From f791e47e1cc99b9236c202014e69a048e0c6eb8c Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Mon, 15 Jul 2013 21:39:35 -0700 Subject: [PATCH 001/331] Stack trace on scalatest failures --- project/SharkBuild.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 71262bf9..485ff0a8 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -62,6 +62,8 @@ object SharkBuild extends Build { javaOptions in test += "-XX:MaxPermSize=512m", javaOptions in test += "-Xmx2g", + testOptions in Test += Tests.Argument("-oF"), // Full stack trace on test failures + testListeners <<= target.map( t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), From 4b8dd1fa7a83fbdf325a8be9018967a97170bccc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 Aug 2013 22:26:43 -0700 Subject: [PATCH 002/331] Added the initial functionality to convert RDDs into Shark in-memory tables. --- src/main/scala/shark/api/DataTypes.java | 81 ++++++++++-- src/main/scala/shark/api/RDDTable.scala | 115 ++++++++++++++++++ .../memstore2/TablePartitionBuilder.scala | 23 ++-- src/main/scala/shark/util/HiveUtils.scala | 77 ++++++++++++ 4 files changed, 273 insertions(+), 23 deletions(-) create mode 100644 src/main/scala/shark/api/RDDTable.scala create mode 100644 src/main/scala/shark/util/HiveUtils.scala diff --git a/src/main/scala/shark/api/DataTypes.java b/src/main/scala/shark/api/DataTypes.java index f8994c7f..1f71eb05 100644 --- a/src/main/scala/shark/api/DataTypes.java +++ b/src/main/scala/shark/api/DataTypes.java @@ -17,8 +17,13 @@ package shark.api; +import java.util.Date; import java.util.HashMap; import java.util.Map; +import java.sql.Timestamp; + +import scala.reflect.ClassManifest; +import scala.reflect.ClassManifest$; import org.apache.hadoop.hive.serde.Constants; @@ -30,19 +35,38 @@ public class DataTypes { // This list of types are defined in a Java class for better interoperability with Shark's // Java APIs. // Primitive types: - public static final DataType BOOLEAN = new DataType("boolean", Constants.BOOLEAN_TYPE_NAME, true); - public static final DataType TINYINT = new DataType("tinyint", Constants.TINYINT_TYPE_NAME, true); - public static final DataType SMALLINT = - new DataType("smallint", Constants.SMALLINT_TYPE_NAME, true); - public static final DataType INT = new DataType("int", Constants.INT_TYPE_NAME, true); - public static final DataType BIGINT = new DataType("bigint", Constants.BIGINT_TYPE_NAME, true); - public static final DataType FLOAT = new DataType("float", Constants.FLOAT_TYPE_NAME, true); - public static final DataType DOUBLE = new DataType("double", Constants.DOUBLE_TYPE_NAME, true); - public static final DataType STRING = new DataType("string", Constants.STRING_TYPE_NAME, true); - public static final DataType TIMESTAMP = - new DataType("timestamp", Constants.TIMESTAMP_TYPE_NAME, true); - public static final DataType DATE = new DataType("date", Constants.DATE_TYPE_NAME, true); - public static final DataType BINARY = new DataType("binary", Constants.BINARY_TYPE_NAME, true); + public static final DataType BOOLEAN = new DataType( + "boolean", Constants.BOOLEAN_TYPE_NAME, true); + + public static final DataType TINYINT = new DataType( + "tinyint", Constants.TINYINT_TYPE_NAME, true); + + public static final DataType SMALLINT = new DataType( + "smallint", Constants.SMALLINT_TYPE_NAME, true); + + public static final DataType INT = new DataType( + "int", Constants.INT_TYPE_NAME, true); + + public static final DataType BIGINT = new DataType( + "bigint", Constants.BIGINT_TYPE_NAME, true); + + public static final DataType FLOAT = new DataType( + "float", Constants.FLOAT_TYPE_NAME, true); + + public static final DataType DOUBLE = new DataType( + "double", Constants.DOUBLE_TYPE_NAME, true); + + public static final DataType STRING = new DataType( + "string", Constants.STRING_TYPE_NAME, true); + + public static final DataType TIMESTAMP = new DataType( + "timestamp", Constants.TIMESTAMP_TYPE_NAME, true); + + public static final DataType DATE = new DataType( + "date", Constants.DATE_TYPE_NAME, true); + + public static final DataType BINARY = new DataType( + "binary", Constants.BINARY_TYPE_NAME, true); // Complex types: // TODO: handle complex types. @@ -82,4 +106,35 @@ public static DataType fromHiveType(String hiveType) throws UnknownDataTypeExcep return type; } } + + public static DataType fromManifest(ClassManifest m) throws UnknownDataTypeException { + if (m.equals(m(Boolean.class)) || m.equals(ClassManifest$.MODULE$.Boolean())) { + return INT; + } else if (m.equals(m(Byte.class)) || m.equals(ClassManifest$.MODULE$.Byte())) { + return TINYINT; + } else if (m.equals(m(Short.class)) || m.equals(ClassManifest$.MODULE$.Short())) { + return SMALLINT; + } else if (m.equals(m(Integer.class)) || m.equals(ClassManifest$.MODULE$.Int())) { + return INT; + } else if (m.equals(m(Long.class)) || m.equals(ClassManifest$.MODULE$.Long())) { + return BIGINT; + } else if (m.equals(m(Float.class)) || m.equals(ClassManifest$.MODULE$.Float())) { + return FLOAT; + } else if (m.equals(m(Double.class)) || m.equals(ClassManifest$.MODULE$.Double())) { + return DOUBLE; + } else if (m.equals(m(String.class))) { + return STRING; + } else if (m.equals(m(Timestamp.class))) { + return TIMESTAMP; + } else if (m.equals(m(Date.class))) { + return DATE; + } else { + throw new UnknownDataTypeException(m.toString()); + } + // TODO: binary data type. + } + + private static ClassManifest m(Class cls) { + return ClassManifest$.MODULE$.fromClass(cls); + } } diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala new file mode 100644 index 00000000..3cb3e988 --- /dev/null +++ b/src/main/scala/shark/api/RDDTable.scala @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +import scala.collection.mutable.ArrayBuffer + +import shark.SharkEnv +import shark.memstore2.{TablePartitionStats, TablePartition, TablePartitionBuilder} +import shark.util.HiveUtils +import spark.RDD + + +class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { + + def saveAsTable(tableName: String, fields: Seq[String]) { + // Get a local copy of the manifests so we don't need to serialize this object. + val manifests = this.manifests + + val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) + + // Create the RDD object. + val rdd = self.mapPartitionsWithIndex { case(partitionIndex, iter) => + val ois = manifests.map(HiveUtils.getJavaPrimitiveObjectInspector) + val builder = new TablePartitionBuilder(ois, 1000000, shouldCompress = false) + + for (p <- iter) { + builder.incrementRowCount() + // TODO: this is not the most efficient code to do the insertion ... + p.productIterator.zipWithIndex.foreach { case (v, i) => + builder.append(i, v.asInstanceOf[Object], ois(i)) + } + } + + statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) + Iterator(builder.build()) + }.persist() + + // Put the table in the metastore. Let's use a fake DML statement. + HiveUtils.createTable(tableName, fields, manifests) + + // Force evaluate to put the data in memory. + SharkEnv.memoryMetadataManager.put(tableName, rdd) + rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + + // Gather the partition statistics. + SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) + } +} + + +object RDDTable { + + private type M[T] = ClassManifest[T] + private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + + def apply[T1: M, T2: M](rdd: RDD[(T1, T2)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2])) + } + + def apply[T1: M, T2: M, T3: M](rdd: RDD[(T1, T2, T3)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3])) + } + + def apply[T1: M, T2: M, T3: M, T4: M](rdd: RDD[(T1, T2, T3, T4)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M](rdd: RDD[(T1, T2, T3, T4, T5)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](rdd: RDD[(T1, T2, T3, T4, T5, T6)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + } +} diff --git a/src/main/scala/shark/memstore2/TablePartitionBuilder.scala b/src/main/scala/shark/memstore2/TablePartitionBuilder.scala index cdd2843d..8614c070 100644 --- a/src/main/scala/shark/memstore2/TablePartitionBuilder.scala +++ b/src/main/scala/shark/memstore2/TablePartitionBuilder.scala @@ -18,10 +18,10 @@ package shark.memstore2 import java.io.{DataInput, DataOutput} -import java.util.{List => JList} + +import scala.collection.JavaConversions._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.StructField import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.io.Writable @@ -33,19 +33,22 @@ import shark.memstore2.column.ColumnBuilder * partition of data into columnar format and to generate a TablePartition. */ class TablePartitionBuilder( - oi: StructObjectInspector, + ois: Seq[ObjectInspector], initialColumnSize: Int, - shouldCompress: Boolean = true) + shouldCompress: Boolean) extends Writable { - var numRows: Long = 0 - val fields: JList[_ <: StructField] = oi.getAllStructFieldRefs + def this(oi: StructObjectInspector, initialColumnSize: Int, shouldCompress: Boolean = true) = { + this(oi.getAllStructFieldRefs.map(_.getFieldObjectInspector), initialColumnSize, shouldCompress) + } + + private var numRows: Long = 0 - val columnBuilders = Array.tabulate[ColumnBuilder[_]](fields.size) { i => - val columnBuilder = ColumnBuilder.create(fields.get(i).getFieldObjectInspector, shouldCompress) + private val columnBuilders: Array[ColumnBuilder[_]] = ois.map { oi => + val columnBuilder = ColumnBuilder.create(oi, shouldCompress) columnBuilder.initialize(initialColumnSize) columnBuilder - } + }.toArray def incrementRowCount() { numRows += 1 @@ -57,7 +60,7 @@ class TablePartitionBuilder( def stats: TablePartitionStats = new TablePartitionStats(columnBuilders.map(_.stats), numRows) - def build: TablePartition = new TablePartition(numRows, columnBuilders.map(_.build)) + def build(): TablePartition = new TablePartition(numRows, columnBuilders.map(_.build())) // We don't use these, but want to maintain Writable interface for SerDe override def write(out: DataOutput) {} diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala new file mode 100644 index 00000000..fc929a31 --- /dev/null +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.util + +import java.util.{ArrayList => JavaArrayList, HashSet => JavaHashSet} +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} +import org.apache.hadoop.hive.ql.plan.{DDLWork, CreateTableDesc} +import org.apache.hadoop.hive.metastore.api.FieldSchema + +import shark.api.DataTypes +import org.apache.hadoop.hive.ql.exec.DDLTask +import org.apache.hadoop.hive.conf.HiveConf + + +private[shark] object HiveUtils { + + private val timestampManfiest = classManifest[java.sql.Timestamp] + private val stringManifest = classManifest[String] + + def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = m match { + case Manifest.Boolean => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case Manifest.Byte => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case Manifest.Short => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case Manifest.Int => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case Manifest.Long => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case Manifest.Float => PrimitiveObjectInspectorFactory.javaFloatObjectInspector + case Manifest.Double => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case Manifest.Unit => PrimitiveObjectInspectorFactory.javaVoidObjectInspector + case `timestampManfiest` => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case `stringManifest` => PrimitiveObjectInspectorFactory.javaStringObjectInspector + } + + /** + * Execute the create table DDL operation against Hive's metastore. + */ + def createTable(tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]]) { + val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => + new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") + } + + // Setup the create table descriptor with necessary information. + val desc = new CreateTableDesc() + desc.setTableName(tableName) + desc.setCols(new JavaArrayList[FieldSchema](schema)) + desc.setTblProps(Map("shark.cache" -> "heap")) + desc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") + desc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") + desc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) + desc.setNumBuckets(-1) + + // Execute the create table against the metastore. + val work = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], desc) + val task = new DDLTask + task.initialize(new HiveConf, null, null) + task.setWork(work) + task.execute(null) + } +} \ No newline at end of file From 019f854d02625d5f87593eeb4bcb2d5d51fe8a36 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 Aug 2013 22:41:04 -0700 Subject: [PATCH 003/331] Error handling for RDDTable. --- src/main/scala/shark/api/RDDTable.scala | 23 +++++++++++++++-------- src/main/scala/shark/util/HiveUtils.scala | 10 ++++++++-- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 3cb3e988..a03622c2 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -27,7 +27,10 @@ import spark.RDD class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { - def saveAsTable(tableName: String, fields: Seq[String]) { + def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { + require(fields.size == this.manifests.size, + "Number of column names != number of fields in the RDD.") + // Get a local copy of the manifests so we don't need to serialize this object. val manifests = this.manifests @@ -50,15 +53,19 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() - // Put the table in the metastore. Let's use a fake DML statement. - HiveUtils.createTable(tableName, fields, manifests) + // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. + if (HiveUtils.createTable(tableName, fields, manifests)) { + // Force evaluate to put the data in memory. + SharkEnv.memoryMetadataManager.put(tableName, rdd) + rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) - // Force evaluate to put the data in memory. - SharkEnv.memoryMetadataManager.put(tableName, rdd) - rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + // Gather the partition statistics. + SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) - // Gather the partition statistics. - SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) + true + } else { + false + } } } diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index fc929a31..d7542f90 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -52,7 +52,11 @@ private[shark] object HiveUtils { /** * Execute the create table DDL operation against Hive's metastore. */ - def createTable(tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]]) { + def createTable( + tableName: String, + columnNames: Seq[String], + columnTypes: Seq[ClassManifest[_]]): Boolean = + { val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") } @@ -72,6 +76,8 @@ private[shark] object HiveUtils { val task = new DDLTask task.initialize(new HiveConf, null, null) task.setWork(work) - task.execute(null) + + // Hive returns 0 if the create table command is executed successfully. + task.execute(null) == 0 } } \ No newline at end of file From e3a4ba67d64b97312020f9ec1b69e90d37a01574 Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Fri, 30 Aug 2013 00:12:17 -0700 Subject: [PATCH 004/331] Add DictEncoding support for Int and Long. 32K uniques using Shorts for encoding --- .../column/CompressedColumnIterator.scala | 9 ++- .../column/CompressionAlgorithm.scala | 27 ++++--- .../CompressedColumnIteratorSuite.scala | 60 +++++++++----- .../column/CompressionAlgorithmSuite.scala | 80 ++++++++++++------- 4 files changed, 109 insertions(+), 67 deletions(-) diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index 7b4e5ab8..59fb5997 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -37,6 +37,7 @@ trait CompressedColumnIterator extends ColumnIterator{ } } + def hasNext = _decoder.hasNext override def current = _current.asInstanceOf[Object] } @@ -81,14 +82,14 @@ class RLDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends Ite class DictDecoder[V] (buffer:ByteBuffer, columnType: ColumnType[_, V]) extends Iterator[V] { - private val _dictionary: Map[Int, V] = { + private val _dictionary: Map[Short, V] = { val size = buffer.getInt() - val d = new HashMap[Int, V]() + val d = new HashMap[Short, V]() var count = 0 while (count < size) { //read text, followed by index val text = columnType.extract(buffer.position(), buffer) - val index = buffer.getInt() + val index = buffer.getShort() d.put(index, text.asInstanceOf[V]) count+= 1 } @@ -98,7 +99,7 @@ class DictDecoder[V] (buffer:ByteBuffer, columnType: ColumnType[_, V]) extends I override def hasNext = buffer.hasRemaining() override def next(): V = { - val index = buffer.getInt() + val index = buffer.getShort() _dictionary.get(index).get } } diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index a26d2ff5..e29478cd 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -139,22 +139,26 @@ class RLE extends CompressionAlgorithm { class DictionaryEncoding extends CompressionAlgorithm { - private val MAX_DICT_SIZE = 4000 - private val _dictionary = new HashMap[Any, Int]() + private val MAX_DICT_SIZE = Short.MaxValue - 1 // 32K unique values allowed + private val _dictionary = new HashMap[Any, Short]() + private val indexSize = 2 // Short + private var _dictionarySize = 0 private var _totalSize = 0 private var _count = 0 - private var _index = 0 + private var _index: Short = 0 private var _overflow = false override def compressionType = DictionaryCompressionType override def supportsType(t: ColumnType[_, _]) = t match { case STRING => true + case LONG => true + case INT => true case _ => false } - private def encode[T](v: T, t: ColumnType[T, _], sizeFunc:T => Int): Int = { + private def encode[T](v: T, t: ColumnType[T, _], sizeFunc:T => Int): Short = { _count += 1 val size = sizeFunc(v) _totalSize += size @@ -164,8 +168,8 @@ class DictionaryEncoding extends CompressionAlgorithm { case Some(index) => index case None => { _dictionary.put(s, _index) - _index += 1 - _dictionarySize += (size + 4) + _index = (1 + _index).toShort + _dictionarySize += (size + indexSize) _index } } @@ -187,7 +191,7 @@ class DictionaryEncoding extends CompressionAlgorithm { * return score between 0 and 1, smaller score imply higher compressibility. */ override def compressionRatio: Double = { - if (_overflow) 1.0 else (_count*4 + dictionarySize) / (_totalSize + 0.0) + if (_overflow) 1.0 else (bufferSize) / (_totalSize + 0.0) } private def writeDictionary[T](compressedBuffer: ByteBuffer, t: ColumnType[T, _]) { @@ -196,15 +200,16 @@ class DictionaryEncoding extends CompressionAlgorithm { //store the dictionary _dictionary.foreach { x => t.append(x._1.asInstanceOf[T], compressedBuffer) - compressedBuffer.putInt(x._2) + compressedBuffer.putShort(x._2) } } private def dictionarySize = _dictionarySize + 4 + private def bufferSize = { _count*indexSize + dictionarySize + 4 + 4 } override def compress[T](b: ByteBuffer, t: ColumnType[T, _]): ByteBuffer = { //build a dictionary of given size - val compressedBuffer = ByteBuffer.allocate(_count*4 + dictionarySize + 4 + 4) + val compressedBuffer = ByteBuffer.allocate(bufferSize) compressedBuffer.order(ByteOrder.nativeOrder()) compressedBuffer.putInt(b.getInt()) compressedBuffer.putInt(compressionType.typeID) @@ -214,11 +219,11 @@ class DictionaryEncoding extends CompressionAlgorithm { while (b.hasRemaining()) { val v = t.extract(b.position(), b) _dictionary.get(v).map { index => - compressedBuffer.putInt(index) + compressedBuffer.putShort(index) } } compressedBuffer.rewind() compressedBuffer } -} \ No newline at end of file +} diff --git a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala index 322de1e1..75a9cb06 100644 --- a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala @@ -36,29 +36,47 @@ class CompressedColumnIteratorSuite extends FunSuite { } test("Dictionary Decompression") { - val b = ByteBuffer.allocate(1024) - b.order(ByteOrder.nativeOrder()) - b.putInt(STRING.typeID) - val dict = new DictionaryEncoding() - - Array(new Text("abc"), new Text("abc"), new Text("efg"), new Text("abc")).foreach { text => - STRING.append(text, b) - dict.gatherStatsForCompressibility(text, STRING) + + def testList[T]( + l: Seq[T], + u: ColumnType[T, _], + expectedDictSize: Int, + compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b) { + + val b = ByteBuffer.allocate(1024 + (3*40*l.size)) + b.order(ByteOrder.nativeOrder()) + b.putInt(u.typeID) + val de = new DictionaryEncoding() + l.foreach { item => + u.append(item.asInstanceOf[T], b) + de.gatherStatsForCompressibility(item, u.asInstanceOf[ColumnType[Any, _]]) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = de.compress(b, u) + val iter = new TestIterator(compressedBuffer, compressedBuffer.getInt()) + l.foreach { x => + iter.next() + assert(compareFunc(iter.current.asInstanceOf[T], x)) + } + assert(false === iter.hasNext) // no extras at the end } - b.limit(b.position()) - b.rewind() - val compressedBuffer = dict.compress(b, STRING) - val iter = new TestIterator(compressedBuffer, compressedBuffer.getInt()) - iter.next() - assert(iter.current.toString().equals("abc")) - iter.next() - assert(iter.current.toString().equals("abc")) - assert(iter.current.toString().equals("abc")) - iter.next() - assert(iter.current.toString().equals("efg")) - iter.next() - assert(iter.current.toString().equals("abc")) + + val iList = Array[Int](10, 10, 20, 10) + val lList = iList.map { i => i.toLong } + val sList = iList.map { i => new Text(i.toString) } + + testList(iList, INT, 2) + testList(lList, LONG, 2) + testList(sList, STRING, 2, (a: Text, b: Text) => a.hashCode == b.hashCode) + + // test at limit of unique values + val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } + val longList = List.concat(iList, alternating, iList) + assert(longList.size === (8 + 2*(Short.MaxValue-1))) + testList(longList, INT, Short.MaxValue - 1) } + } class TestIterator(val buffer: ByteBuffer, val columnType: ColumnType[_,_]) diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index 83d3b717..91e40e06 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -139,38 +139,56 @@ class CompressionAlgorithmSuite extends FunSuite { } test("Dictionary Encoding") { - val b = ByteBuffer.allocate(1024) - b.order(ByteOrder.nativeOrder()) - b.putInt(STRING.typeID) - val de = new DictionaryEncoding() - Array[Text](new Text("abc"), - new Text("abc"), - new Text("efg"), - new Text("abc")).foreach { text => - STRING.append(text, b) - de.gatherStatsForCompressibility(text, STRING) + + def testList[T]( + l: Seq[T], + u: ColumnType[T, _], + expectedDictSize: Int, + compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b) { + + val b = ByteBuffer.allocate(1024 + (3*40*l.size)) + b.order(ByteOrder.nativeOrder()) + b.putInt(u.typeID) + val de = new DictionaryEncoding() + l.foreach { item => + u.append(item.asInstanceOf[T], b) + de.gatherStatsForCompressibility(item, u.asInstanceOf[ColumnType[Any, _]]) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = de.compress(b, u) + assert(compressedBuffer.getInt() === u.typeID) + assert(compressedBuffer.getInt() === DictionaryCompressionType.typeID) + assert(compressedBuffer.getInt() === expectedDictSize) //dictionary size + val dictionary = new HashMap[Short, T]() + var count = 0 + while (count < expectedDictSize) { + val v = u.extract(compressedBuffer.position(), compressedBuffer) + val index = compressedBuffer.getShort() + dictionary.put(index, u.clone(v)) + count += 1 + } + assert(dictionary.get(0).get.equals(l(0))) + assert(dictionary.get(1).get.equals(l(2))) + l.foreach { x => + val y = dictionary.get(compressedBuffer.getShort()).get + assert(compareFunc(y, x)) } - b.limit(b.position()) - b.rewind() - val compressedBuffer = de.compress(b, STRING) - assert(compressedBuffer.getInt() == STRING.typeID) - assert(compressedBuffer.getInt() == DictionaryCompressionType.typeID) - assert(compressedBuffer.getInt() == 2) //dictionary size - val dictionary = new HashMap[Int, Text]() - var count = 0 - while (count < 2) { - val v = STRING.extract(compressedBuffer.position(), compressedBuffer) - val index = compressedBuffer.getInt() - dictionary.put(index, v) - count += 1 } - assert(dictionary.get(0).get.equals(new Text("abc"))) - assert(dictionary.get(1).get.equals(new Text("efg"))) - //read the next 4 items - assert(compressedBuffer.getInt() == 0) - assert(compressedBuffer.getInt() == 0) - assert(compressedBuffer.getInt() == 1) - assert(compressedBuffer.getInt() == 0) + + val iList = Array[Int](10, 10, 20, 10) + val lList = iList.map { i => i.toLong } + val sList = iList.map { i => new Text(i.toString) } + + testList(iList, INT, 2) + testList(lList, LONG, 2) + testList(sList, STRING, 2, (a: Text, b: Text) => a.hashCode == b.hashCode) + + // test at limit of unique values + val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } + val longList = List.concat(iList, alternating, iList) + assert(longList.size === (8 + 2*(Short.MaxValue-1))) + testList(longList, INT, Short.MaxValue - 1) } test("RLE region") { @@ -192,4 +210,4 @@ class CompressionAlgorithmSuite extends FunSuite { assert(newBuffer.getInt() == RLECompressionType.typeID) } -} \ No newline at end of file +} From 4585bb81a8336a61005c5e1609b42f59c27dfc21 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 3 Sep 2013 17:30:06 -0700 Subject: [PATCH 005/331] Removed no-longer-needed repos --- project/SharkBuild.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 566917e8..9c9aaa32 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -59,8 +59,6 @@ object SharkBuild extends Build { retrieveManaged := true, resolvers ++= Seq( "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - "Spray Repository" at "http://repo.spray.cc/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", "Local Maven" at Path.userHome.asFile.toURI.toURL + ".m2/repository" ), From e17faddec1efed426d18c800b3cd016b0d925e44 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 4 Sep 2013 16:24:51 -0700 Subject: [PATCH 006/331] Capitalize CacheType enums --- .../scala/shark/execution/TableScanOperator.scala | 4 ++-- src/main/scala/shark/memstore2/CacheType.scala | 10 +++++----- .../scala/shark/parse/SharkSemanticAnalyzer.scala | 14 +++++++------- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 9fda4702..2c82762f 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -119,7 +119,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO val cacheMode = CacheType.fromString( tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) - if (cacheMode == CacheType.heap) { + if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). val rdd = SharkEnv.memoryMetadataManager.get(tableKey).getOrElse { logError("""|Table %s not found in block manager. @@ -129,7 +129,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } logInfo("Loading table " + tableKey + " from Spark block manager") createPrunedRdd(tableKey, rdd) - } else if (cacheMode == CacheType.tachyon) { + } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala index 13115415..deaf8b7f 100644 --- a/src/main/scala/shark/memstore2/CacheType.scala +++ b/src/main/scala/shark/memstore2/CacheType.scala @@ -21,19 +21,19 @@ package shark.memstore2 object CacheType extends Enumeration { type CacheType = Value - val none, heap, tachyon = Value + val NONE, HEAP, TACHYON = Value - def shouldCache(c: CacheType): Boolean = (c != none) + def shouldCache(c: CacheType): Boolean = (c != NONE) /** Get the cache type object from a string representation. */ def fromString(name: String): CacheType = { if (name == null || name == "") { - none + NONE } else if (name.toLowerCase == "true") { - heap + HEAP } else { try { - withName(name.toLowerCase) + withName(name.toUpperCase) } catch { case e: java.util.NoSuchElementException => throw new InvalidCacheTypeException(name) } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 199102bc..86a20092 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -76,7 +76,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with //TODO: can probably reuse Hive code for this // analyze create table command - var cacheMode = CacheType.none + var cacheMode = CacheType.NONE var isCTAS = false var shouldReset = false @@ -101,13 +101,13 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } else { val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) val cacheType = CacheType.fromString(td.getTblProps().get("shark.cache")) - if (cacheType == CacheType.heap || + if (cacheType == CacheType.HEAP || (td.getTableName.endsWith("_cached") && checkTableName)) { - cacheMode = CacheType.heap + cacheMode = CacheType.HEAP td.getTblProps().put("shark.cache", cacheMode.toString) - } else if (cacheType == CacheType.tachyon || + } else if (cacheType == CacheType.TACHYON || (td.getTableName.endsWith("_tachyon") && checkTableName)) { - cacheMode = CacheType.tachyon + cacheMode = CacheType.TACHYON td.getTblProps().put("shark.cache", cacheMode.toString) } @@ -191,7 +191,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with cachedTableName, storageLevel, _resSchema.size, // numColumns - cacheMode == CacheType.tachyon, // use tachyon + cacheMode == CacheType.TACHYON, // use tachyon useUnionRDD) } else { throw new SemanticException( @@ -215,7 +215,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.getTableDesc.getTableName, storageLevel, _resSchema.size, // numColumns - cacheMode == CacheType.tachyon, // use tachyon + cacheMode == CacheType.TACHYON, // use tachyon false) } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) From 7d097a5dc8d7297805f5c23343a7534ba0bc71c7 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 4 Sep 2013 16:29:44 -0700 Subject: [PATCH 007/331] Some comments for CacheTypes and small refactoring. --- .../shark/execution/TableScanOperator.scala | 3 +-- .../scala/shark/memstore2/CacheType.scala | 14 ++++++++++-- .../shark/parse/SharkSemanticAnalyzer.scala | 22 +++++++++++-------- 3 files changed, 26 insertions(+), 13 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 2c82762f..9115bac7 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -114,9 +114,8 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // There are three places we can load the table from. // 1. Tachyon table - // 2. Spark heap (block manager) + // 2. Spark heap (block manager), accessed through the Shark MemoryMetadataManager // 3. Hive table on HDFS (or other Hadoop storage) - val cacheMode = CacheType.fromString( tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) if (cacheMode == CacheType.HEAP) { diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala index deaf8b7f..a84f4df9 100644 --- a/src/main/scala/shark/memstore2/CacheType.scala +++ b/src/main/scala/shark/memstore2/CacheType.scala @@ -17,9 +17,17 @@ package shark.memstore2 - +/* + * Enumerations and static helper functions for cache types that can be used by Shark. + */ object CacheType extends Enumeration { + /* The three CacheTypes: + * - NONE: On-disk storage (e.g. a Hive table that is stored in HDFS ). + * - HEAP: refers to Spark's block manager, which coordinates in-memory and on-disk RDD storage. + * - TACHYON: A distributed storage system that manages an in-memory cache for sharing files and + * RDDs across cluster frameworks. + */ type CacheType = Value val NONE, HEAP, TACHYON = Value @@ -33,6 +41,7 @@ object CacheType extends Enumeration { HEAP } else { try { + // Try to use Scala's Enumeration::withName() to interpret 'name'. withName(name.toUpperCase) } catch { case e: java.util.NoSuchElementException => throw new InvalidCacheTypeException(name) @@ -40,5 +49,6 @@ object CacheType extends Enumeration { } } - class InvalidCacheTypeException(name: String) extends Exception("Invalid cache type " + name) + class InvalidCacheTypeException(name: String) + extends Exception("Invalid string representation of cache type " + name) } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 86a20092..fa75ee68 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -20,6 +20,7 @@ package shark.parse import java.lang.reflect.Method import java.util.ArrayList import java.util.{List => JavaList} +import java.util.{Map => JavaMap} import scala.collection.JavaConversions._ @@ -95,27 +96,30 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // If the table descriptor can be null if the CTAS has an // "if not exists" condition. - val td = getParseContext.getQB.getTableDesc - if (!isCTAS || td == null) { + val createTableDesc = getParseContext.getQB.getTableDesc + if (!isCTAS || createTableDesc == null) { return } else { val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) - val cacheType = CacheType.fromString(td.getTblProps().get("shark.cache")) + // Note: the CreateTableDesc's table properties are Java Maps, but the TableDesc's table + // properties, which are used during execution, are Java Properties. + val createTableProperties: JavaMap[String, String] = createTableDesc.getTblProps() + val cacheType = CacheType.fromString(createTableProperties.get("shark.cache")) if (cacheType == CacheType.HEAP || - (td.getTableName.endsWith("_cached") && checkTableName)) { + (createTableDesc.getTableName.endsWith("_cached") && checkTableName)) { cacheMode = CacheType.HEAP - td.getTblProps().put("shark.cache", cacheMode.toString) + createTableProperties.put("shark.cache", cacheMode.toString) } else if (cacheType == CacheType.TACHYON || - (td.getTableName.endsWith("_tachyon") && checkTableName)) { + (createTableDesc.getTableName.endsWith("_tachyon") && checkTableName)) { cacheMode = CacheType.TACHYON - td.getTblProps().put("shark.cache", cacheMode.toString) + createTableProperties.put("shark.cache", cacheMode.toString) } if (CacheType.shouldCache(cacheMode)) { - td.setSerName(classOf[ColumnarSerDe].getName) + createTableDesc.setSerName(classOf[ColumnarSerDe].getName) } - qb.setTableDesc(td) + qb.setTableDesc(createTableDesc) shouldReset = true } } else { From 4ae78cf641713b7ebeca4b88a990b9f5250ed239 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 4 Sep 2013 18:01:20 -0700 Subject: [PATCH 008/331] Add a "shark.cache.type" flag in TBLPROPERTIES This allows the user to pass a string representation of a supported cache type (heap, tachyon). The flag addition also fixes a bug where passing 'shark.cache'='false' causes an InvalidCacheTypeException to be thrown. --- .../shark/execution/TableScanOperator.scala | 2 +- .../scala/shark/memstore2/CacheType.scala | 6 ++--- .../shark/parse/SharkSemanticAnalyzer.scala | 22 ++++++++++++++----- src/test/scala/shark/SQLSuite.scala | 14 ++++++++++-- 4 files changed, 32 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 9115bac7..ef33d275 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -117,7 +117,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // 2. Spark heap (block manager), accessed through the Shark MemoryMetadataManager // 3. Hive table on HDFS (or other Hadoop storage) val cacheMode = CacheType.fromString( - tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) + tableDesc.getProperties().get("shark.cache.type").asInstanceOf[String]) if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). val rdd = SharkEnv.memoryMetadataManager.get(tableKey).getOrElse { diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala index a84f4df9..3c13b2bb 100644 --- a/src/main/scala/shark/memstore2/CacheType.scala +++ b/src/main/scala/shark/memstore2/CacheType.scala @@ -18,11 +18,11 @@ package shark.memstore2 /* - * Enumerations and static helper functions for cache types that can be used by Shark. + * Enumerations and static helper functions for caches supported by Shark. */ object CacheType extends Enumeration { - /* The three CacheTypes: + /* The CacheTypes: * - NONE: On-disk storage (e.g. a Hive table that is stored in HDFS ). * - HEAP: refers to Spark's block manager, which coordinates in-memory and on-disk RDD storage. * - TACHYON: A distributed storage system that manages an in-memory cache for sharing files and @@ -37,8 +37,6 @@ object CacheType extends Enumeration { def fromString(name: String): CacheType = { if (name == null || name == "") { NONE - } else if (name.toLowerCase == "true") { - HEAP } else { try { // Try to use Scala's Enumeration::withName() to interpret 'name'. diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index fa75ee68..9a6c17ed 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -104,15 +104,27 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // Note: the CreateTableDesc's table properties are Java Maps, but the TableDesc's table // properties, which are used during execution, are Java Properties. val createTableProperties: JavaMap[String, String] = createTableDesc.getTblProps() - val cacheType = CacheType.fromString(createTableProperties.get("shark.cache")) - if (cacheType == CacheType.HEAP || + + // There are two cases that will enable caching: + // 1) Table name includes "_cached" or "_tachyon". + // 2) The "shark.cache" table property is true. + // - In this case, the cache type can be specified by a "shark.cache.type" table + // property + val hasCacheFlag = createTableProperties.getOrElse("shark.cache", "false").toBoolean + if (hasCacheFlag) { + // If "shark.cache" is true, then default to CacheType.HEAP. + cacheMode = CacheType.fromString( + createTableProperties.getOrElse("shark.cache.type", "heap")) + } + // Continue planning based on the 'cacheMode' read. + if (cacheMode == CacheType.HEAP || (createTableDesc.getTableName.endsWith("_cached") && checkTableName)) { cacheMode = CacheType.HEAP - createTableProperties.put("shark.cache", cacheMode.toString) - } else if (cacheType == CacheType.TACHYON || + createTableProperties.put("shark.cache.type", cacheMode.toString) + } else if (cacheMode == CacheType.TACHYON || (createTableDesc.getTableName.endsWith("_tachyon") && checkTableName)) { cacheMode = CacheType.TACHYON - createTableProperties.put("shark.cache", cacheMode.toString) + createTableProperties.put("shark.cache.type", cacheMode.toString) } if (CacheType.shouldCache(cacheMode)) { diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 9751bcb3..df9a667d 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -50,7 +50,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // test_null sc.runSql("drop table if exists test_null") sc.runSql("CREATE TABLE test_null (key INT, val STRING)") - sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' INTO TABLE test_null") + sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' + INTO TABLE test_null") sc.runSql("drop table if exists test_null_cached") sc.runSql("CREATE TABLE test_null_cached AS SELECT * FROM test_null") @@ -257,7 +258,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { expectSql("select count(*) from foo_cached", "0") } - test("create cached table with table properties") { + test("create cached table with 'shark.cache' flag in table properties") { sc.runSql("drop table if exists ctas_tbl_props") sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='true') as select * from test""") @@ -265,6 +266,15 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") } + test("default to Hive table creation when 'shark.cache' flag is false in table properties") { + sc.runSql("drop table if exists ctas_tbl_props_should_not_be_cached") + sc.runSql(""" + CREATE TABLE ctas_tbl_props_result_should_not_be_cached + TBLPROPERTIES ('shark.cache'='false') + AS select * from test""") + assert(!SharkEnv.memoryMetadataManager.contains("ctas_tbl_props_should_not_be_cached")) + } + test("cached tables with complex types") { sc.runSql("drop table if exists test_complex_types") sc.runSql("drop table if exists test_complex_types_cached") From 31787ec4fb0ee66ee7c68a157f9fff81c2e5344e Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 4 Sep 2013 18:31:56 -0700 Subject: [PATCH 009/331] Remove "shark.cache.type" property. Handle "shark.cache"="false". --- .../shark/execution/TableScanOperator.scala | 2 +- src/main/scala/shark/memstore2/CacheType.scala | 7 +++++-- .../shark/parse/SharkSemanticAnalyzer.scala | 16 +++++----------- src/test/scala/shark/SQLSuite.scala | 4 ++-- 4 files changed, 13 insertions(+), 16 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index ef33d275..9115bac7 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -117,7 +117,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // 2. Spark heap (block manager), accessed through the Shark MemoryMetadataManager // 3. Hive table on HDFS (or other Hadoop storage) val cacheMode = CacheType.fromString( - tableDesc.getProperties().get("shark.cache.type").asInstanceOf[String]) + tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). val rdd = SharkEnv.memoryMetadataManager.get(tableKey).getOrElse { diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala index 3c13b2bb..b3117c4f 100644 --- a/src/main/scala/shark/memstore2/CacheType.scala +++ b/src/main/scala/shark/memstore2/CacheType.scala @@ -22,7 +22,8 @@ package shark.memstore2 */ object CacheType extends Enumeration { - /* The CacheTypes: + /* + * The CacheTypes: * - NONE: On-disk storage (e.g. a Hive table that is stored in HDFS ). * - HEAP: refers to Spark's block manager, which coordinates in-memory and on-disk RDD storage. * - TACHYON: A distributed storage system that manages an in-memory cache for sharing files and @@ -35,8 +36,10 @@ object CacheType extends Enumeration { /** Get the cache type object from a string representation. */ def fromString(name: String): CacheType = { - if (name == null || name == "") { + if (name == null || name == "" || name.toLowerCase == "false") { NONE + } else if (name.toLowerCase == "true") { + HEAP } else { try { // Try to use Scala's Enumeration::withName() to interpret 'name'. diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 9a6c17ed..2dfbc695 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -107,24 +107,18 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // There are two cases that will enable caching: // 1) Table name includes "_cached" or "_tachyon". - // 2) The "shark.cache" table property is true. - // - In this case, the cache type can be specified by a "shark.cache.type" table - // property - val hasCacheFlag = createTableProperties.getOrElse("shark.cache", "false").toBoolean - if (hasCacheFlag) { - // If "shark.cache" is true, then default to CacheType.HEAP. - cacheMode = CacheType.fromString( - createTableProperties.getOrElse("shark.cache.type", "heap")) - } + // 2) The "shark.cache" table property is "true", or the string representation of a supported + // cache mode (heap, Tachyon). + cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) // Continue planning based on the 'cacheMode' read. if (cacheMode == CacheType.HEAP || (createTableDesc.getTableName.endsWith("_cached") && checkTableName)) { cacheMode = CacheType.HEAP - createTableProperties.put("shark.cache.type", cacheMode.toString) + createTableProperties.put("shark.cache", cacheMode.toString) } else if (cacheMode == CacheType.TACHYON || (createTableDesc.getTableName.endsWith("_tachyon") && checkTableName)) { cacheMode = CacheType.TACHYON - createTableProperties.put("shark.cache.type", cacheMode.toString) + createTableProperties.put("shark.cache", cacheMode.toString) } if (CacheType.shouldCache(cacheMode)) { diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index df9a667d..1d2190c4 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -50,8 +50,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // test_null sc.runSql("drop table if exists test_null") sc.runSql("CREATE TABLE test_null (key INT, val STRING)") - sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' - INTO TABLE test_null") + sc.runSql("""LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' + INTO TABLE test_null""") sc.runSql("drop table if exists test_null_cached") sc.runSql("CREATE TABLE test_null_cached AS SELECT * FROM test_null") From ff37f6c7955daf4b6e6000c73f0a921fb00c0117 Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Thu, 5 Sep 2013 17:05:39 -0700 Subject: [PATCH 010/331] def hasNext is now part of the ColumnIterator interface. Test iteration across column types for both RLE and Dict. Test non-selection of compression algos. Test and add Long Columns to RLE supported types. --- .../memstore2/column/ColumnIterator.scala | 2 + .../column/CompressedColumnIterator.scala | 10 +- .../column/CompressionAlgorithm.scala | 5 +- .../column/NullableColumnIterator.scala | 8 + .../CompressedColumnIteratorSuite.scala | 149 ++++++++++++------ .../column/CompressionAlgorithmSuite.scala | 22 ++- .../column/NullableColumnIteratorSuite.scala | 47 ++++-- 7 files changed, 169 insertions(+), 74 deletions(-) diff --git a/src/main/scala/shark/memstore2/column/ColumnIterator.scala b/src/main/scala/shark/memstore2/column/ColumnIterator.scala index 5c9b267c..2126654f 100644 --- a/src/main/scala/shark/memstore2/column/ColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/ColumnIterator.scala @@ -35,6 +35,8 @@ trait ColumnIterator { computeNext() } + def hasNext: Boolean + def computeNext(): Unit // Should be implemented as a read-only operation by the ColumnIterator diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index 59fb5997..a8135f42 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -38,6 +38,7 @@ trait CompressedColumnIterator extends ColumnIterator{ } def hasNext = _decoder.hasNext + override def current = _current.asInstanceOf[Object] } @@ -88,9 +89,11 @@ class DictDecoder[V] (buffer:ByteBuffer, columnType: ColumnType[_, V]) extends I var count = 0 while (count < size) { //read text, followed by index - val text = columnType.extract(buffer.position(), buffer) + val text = columnType.newWritable + columnType.extractInto(buffer.position(), buffer, text) val index = buffer.getShort() d.put(index, text.asInstanceOf[V]) + count+= 1 } d @@ -100,6 +103,9 @@ class DictDecoder[V] (buffer:ByteBuffer, columnType: ColumnType[_, V]) extends I override def next(): V = { val index = buffer.getShort() - _dictionary.get(index).get + _dictionary.get(index) match { + case Some(s) => s.asInstanceOf[V] + case None => throw new Exception("Bad index into Dictionary") + } } } diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index e29478cd..03fad650 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -69,7 +69,7 @@ class RLE extends CompressionAlgorithm { override def supportsType(t: ColumnType[_, _]) = { t match { - case INT | STRING | SHORT | BYTE | BOOLEAN => true + case LONG | INT | STRING | SHORT | BYTE | BOOLEAN => true case _ => false } } @@ -189,9 +189,10 @@ class DictionaryEncoding extends CompressionAlgorithm { /** * return score between 0 and 1, smaller score imply higher compressibility. + * return Double.MaxValue to indicate overflow so that dict does not get chosen. */ override def compressionRatio: Double = { - if (_overflow) 1.0 else (bufferSize) / (_totalSize + 0.0) + if (_overflow) Double.MaxValue else (bufferSize) / (_totalSize + 0.0) } private def writeDictionary[T](compressedBuffer: ByteBuffer, t: ColumnType[T, _]) { diff --git a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala index 66a3adfa..3ab34a91 100644 --- a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala @@ -42,6 +42,14 @@ class NullableColumnIterator(delegate: ColumnIterator, buffer: ByteBuffer) exten _pos += 1 } + override def hasNext: Boolean = { + if (_nulls < _nullCount) { + true + } else { + delegate.hasNext + } + } + def current: Object = { if (_isNull) null else delegate.current } diff --git a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala index 75a9cb06..12a66f98 100644 --- a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala @@ -5,76 +5,121 @@ import java.nio.ByteOrder import org.scalatest.FunSuite import org.apache.hadoop.io.Text +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import shark.memstore2.column.Implicits._ class CompressedColumnIteratorSuite extends FunSuite { - - test("RLE Decompression") { - val b = ByteBuffer.allocate(1024) + + val booleanList = Array[Boolean](true, true, false, true) + val byteList = Array[Byte](10, 10, 20, 10) + val shortList = byteList.map { i => (Short.MaxValue - i).toShort } + val iList = byteList.map { i => Int.MaxValue - i.toInt } + val lList = iList.map { i => Long.MaxValue - i.toLong } + val sList = iList.map { i => new Text(i.toString) } + + /** Generic tester across types and encodings + * + */ + def testList[T, W]( + l: Seq[T], + u: ColumnType[T, _], + algo: CompressionAlgorithm, + compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b, + shouldNotCompress: Boolean = false) { + + val b = ByteBuffer.allocate(1024 + (3*40*l.size)) b.order(ByteOrder.nativeOrder()) - b.putInt(STRING.typeID) - val rle = new RLE() - - Array(new Text("abc"), new Text("abc"), new Text("efg"), new Text("abc")).foreach { text => - STRING.append(text, b) - rle.gatherStatsForCompressibility(text, STRING) + b.putInt(u.typeID) + l.foreach { item => + u.append(item.asInstanceOf[T], b) + algo.gatherStatsForCompressibility(item, u.asInstanceOf[ColumnType[Any, _]]) } b.limit(b.position()) b.rewind() - val compressedBuffer = rle.compress(b, STRING) - val iter = new TestIterator(compressedBuffer, compressedBuffer.getInt()) - iter.next() - assert(iter.current.toString().equals("abc")) - iter.next() - assert(iter.current.toString().equals("abc")) - assert(iter.current.toString().equals("abc")) - iter.next() - assert(iter.current.toString().equals("efg")) - iter.next() - assert(iter.current.toString().equals("abc")) - } - - test("Dictionary Decompression") { - - def testList[T]( - l: Seq[T], - u: ColumnType[T, _], - expectedDictSize: Int, - compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b) { - - val b = ByteBuffer.allocate(1024 + (3*40*l.size)) - b.order(ByteOrder.nativeOrder()) - b.putInt(u.typeID) - val de = new DictionaryEncoding() - l.foreach { item => - u.append(item.asInstanceOf[T], b) - de.gatherStatsForCompressibility(item, u.asInstanceOf[ColumnType[Any, _]]) - } - b.limit(b.position()) - b.rewind() - val compressedBuffer = de.compress(b, u) + val compressedBuffer = algo.compress(b, u) + if (shouldNotCompress) { + assert(algo.compressionRatio >= 1.0) + info("CompressionRatio " + algo.compressionRatio) + } else { + val iter = new TestIterator(compressedBuffer, compressedBuffer.getInt()) + + val oi: ObjectInspector = u match { + case BOOLEAN => PrimitiveObjectInspectorFactory.writableBooleanObjectInspector + case BYTE => PrimitiveObjectInspectorFactory.writableByteObjectInspector + case SHORT => PrimitiveObjectInspectorFactory.writableShortObjectInspector + case INT => PrimitiveObjectInspectorFactory.writableIntObjectInspector + case LONG => PrimitiveObjectInspectorFactory.writableLongObjectInspector + case STRING => PrimitiveObjectInspectorFactory.writableStringObjectInspector + } + l.foreach { x => iter.next() - assert(compareFunc(iter.current.asInstanceOf[T], x)) + assert(compareFunc(u.get(iter.current, oi), x)) + // assert(u.get(iter.current, oi) === x) } assert(false === iter.hasNext) // no extras at the end } + } + + test("RLE Decompression Boolean") { + testList(booleanList, BOOLEAN, new RLE()) + } + + test("RLE Decompression Byte") { + testList(byteList, BYTE, new RLE()) + } + + test("RLE Decompression Short") { + testList(shortList, SHORT, new RLE()) + } - val iList = Array[Int](10, 10, 20, 10) - val lList = iList.map { i => i.toLong } - val sList = iList.map { i => new Text(i.toString) } + test("RLE Decompression Int") { + testList(iList, INT, new RLE()) + } + + test("RLE Decompression Long") { + testList(lList, LONG, new RLE()) + } + + test("RLE Decompression String") { + testList(sList, STRING, new RLE(), (a: Text, b: Text) => a.hashCode == b.hashCode) + } + + test("Dictionary Decompression Int") { + testList(iList, INT, new DictionaryEncoding()) + } + + test("Dictionary Decompression Long") { + testList(lList, LONG, new DictionaryEncoding()) + } + + test("Dictionary Decompression String") { + testList(sList, STRING, new DictionaryEncoding(), (a: Text, b: Text) => a.hashCode == b.hashCode) + } + + test("Dictionary Decompression at limit of unique values") { + val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } + val iiList = byteList.map { i => i.toInt } + val hugeList = List.concat(iiList, alternating, iiList) + assert(hugeList.size === (8 + 2*(Short.MaxValue-1))) + testList(hugeList, INT, new DictionaryEncoding()) + } - testList(iList, INT, 2) - testList(lList, LONG, 2) - testList(sList, STRING, 2, (a: Text, b: Text) => a.hashCode == b.hashCode) + test("Dictionary Decompression - should not compress") { + val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } + val hugeList = List.concat(iList, alternating, iList) + assert(hugeList.size === (8 + 2*(Short.MaxValue-1))) + testList(hugeList, INT, new DictionaryEncoding(), (a: Int, b: Int) => a == b, true) + } - // test at limit of unique values + test("RLE - should not compress") { val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } - val longList = List.concat(iList, alternating, iList) - assert(longList.size === (8 + 2*(Short.MaxValue-1))) - testList(longList, INT, Short.MaxValue - 1) + val hugeList = List.concat(iList, alternating, iList) + assert(hugeList.size === (8 + 2*(Short.MaxValue-1))) + testList(hugeList, INT, new RLE(), (a: Int, b: Int) => a == b, true) } } diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index 91e40e06..00218547 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -77,7 +77,7 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 1) } - test("RLE perfect encoding") { + test("RLE perfect encoding Int") { val b = ByteBuffer.allocate(4008) b.order(ByteOrder.nativeOrder()) b.putInt(INT.typeID) @@ -95,6 +95,24 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 1000) } + test("RLE perfect encoding Long") { + val b = ByteBuffer.allocate(8008) + b.order(ByteOrder.nativeOrder()) + b.putInt(LONG.typeID) + val rle = new RLE() + Range(0,1000).foreach { x => + b.putLong(Long.MaxValue - 6) + rle.gatherStatsForCompressibility(Long.MaxValue - 6, LONG) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = rle.compress(b, LONG) + assert(compressedBuffer.getInt() == LONG.typeID) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + assert(compressedBuffer.getLong() == Long.MaxValue - 6) + assert(compressedBuffer.getInt() == 1000) + } + test("RLE mixture") { val b = ByteBuffer.allocate(4008) b.order(ByteOrder.nativeOrder()) @@ -177,7 +195,7 @@ class CompressionAlgorithmSuite extends FunSuite { } val iList = Array[Int](10, 10, 20, 10) - val lList = iList.map { i => i.toLong } + val lList = iList.map { i => Long.MaxValue - i.toLong } val sList = iList.map { i => new Text(i.toString) } testList(iList, INT, 2) diff --git a/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala index b5d210eb..f0eb9f95 100644 --- a/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala @@ -25,6 +25,7 @@ class NullableColumnIteratorSuite extends FunSuite { val b = c.build() val i = ColumnIterator.newIterator(b) Range(0, a.length).foreach { x => + if (x > 0) assert(true === i.hasNext) i.next() val v = i.current if (a(x) == null) { @@ -33,7 +34,9 @@ class NullableColumnIteratorSuite extends FunSuite { assert(v.toString == a(x).toString) } } + assert(false === i.hasNext) } + test("Iterate Strings") { val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector val c = ColumnBuilder.create(oi) @@ -58,23 +61,35 @@ class NullableColumnIteratorSuite extends FunSuite { assert(i.current.toString() == "Abcdz") i.next() assert(i.current == null) + assert(false === i.hasNext) } test("Iterate Ints") { - val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector - val c = ColumnBuilder.create(oi) - c.initialize(4) - c.append(123.asInstanceOf[Object],oi) - c.append(null, oi) - c.append(null, oi) - c.append(56.asInstanceOf[Object], oi) - val b = c.build() - val i = ColumnIterator.newIterator(b) - i.next() - assert(i.current.asInstanceOf[IntWritable].get() == 123) - i.next() - assert(i.current == null) - i.next() - assert(i.current == null) + def testList(l: Seq[AnyRef]) { + val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector + val c = ColumnBuilder.create(oi) + c.initialize(l.size) + + l.foreach { item => + c.append(item.asInstanceOf[AnyRef], oi) + } + + val b = c.build() + val i = ColumnIterator.newIterator(b) + + l.foreach { x => + i.next() + if (x == null) { + assert(i.current === x) + } else { + assert(i.current.asInstanceOf[IntWritable].get === x) + } + } + assert(false === i.hasNext) + } + + testList(List(null, null, 123.asInstanceOf[AnyRef])) + testList(List(123.asInstanceOf[AnyRef], 4.asInstanceOf[AnyRef], null)) + testList(List(null)) } -} \ No newline at end of file +} From 3293a851946f5a8604e56c21ce36ae489ea2a545 Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Thu, 5 Sep 2013 17:09:03 -0700 Subject: [PATCH 011/331] override def hasNext --- .../scala/shark/memstore2/column/CompressedColumnIterator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index a8135f42..4315b2e6 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -37,7 +37,7 @@ trait CompressedColumnIterator extends ColumnIterator{ } } - def hasNext = _decoder.hasNext + override def hasNext = _decoder.hasNext override def current = _current.asInstanceOf[Object] } From 1cad87aab33e2e0d35468f4006322e80931a5332 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 6 Sep 2013 16:22:52 +0800 Subject: [PATCH 012/331] Added more documentation to the compression code and slightly simplified the dictionary encoding algorithm. --- .../memstore2/column/ColumnBuilder.scala | 2 +- .../memstore2/column/ColumnIterator.scala | 51 +++--- .../shark/memstore2/column/ColumnType.scala | 97 ++++++---- .../column/CompressedColumnIterator.scala | 22 +-- .../column/CompressionAlgorithm.scala | 168 ++++++++++-------- .../column/MemoryStoreException.scala | 21 +++ .../column/NullableColumnIterator.scala | 14 +- .../CompressedColumnIteratorSuite.scala | 119 ++++++------- .../column/CompressionAlgorithmSuite.scala | 27 +-- .../column/NullableColumnBuilderSuite.scala | 4 +- 10 files changed, 300 insertions(+), 225 deletions(-) create mode 100644 src/main/scala/shark/memstore2/column/MemoryStoreException.scala diff --git a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala index 375ec244..2bb34404 100644 --- a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala +++ b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala @@ -138,7 +138,7 @@ object ColumnBuilder { case PrimitiveCategory.BINARY => new BinaryColumnBuilder // TODO: add decimal column. - case _ => throw new Exception( + case _ => throw new MemoryStoreException( "Invalid primitive object inspector category" + columnOi.getCategory) } } diff --git a/src/main/scala/shark/memstore2/column/ColumnIterator.scala b/src/main/scala/shark/memstore2/column/ColumnIterator.scala index 2126654f..e0ca8520 100644 --- a/src/main/scala/shark/memstore2/column/ColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/ColumnIterator.scala @@ -17,8 +17,7 @@ package shark.memstore2.column -import java.nio.ByteBuffer -import java.nio.ByteOrder +import java.nio.{ByteBuffer, ByteOrder} trait ColumnIterator { @@ -27,6 +26,9 @@ trait ColumnIterator { def init() {} + /** + * Produces the next element of this iterator. + */ def next() { if (!_initialized) { init() @@ -35,12 +37,20 @@ trait ColumnIterator { computeNext() } + /** + * Tests whether this iterator can provide another element. + */ def hasNext: Boolean - def computeNext(): Unit + /** + * Compute the next element so it is ready to be fetched using the current function. + */ + def computeNext() - // Should be implemented as a read-only operation by the ColumnIterator - // Can be called any number of times + /** + * Return the current element. The operation should have no side-effect, i.e. it can be invoked + * multiple times returning the same value. + */ def current: Object } @@ -51,25 +61,25 @@ abstract class DefaultColumnIterator[T, V](val buffer: ByteBuffer, val columnTyp object Implicits { implicit def intToCompressionType(i: Int): CompressionType = i match { - case -1 => DefaultCompressionType - case 0 => RLECompressionType - case 1 => DictionaryCompressionType + case DefaultCompressionType.typeID => DefaultCompressionType + case RLECompressionType.typeID => RLECompressionType + case DictionaryCompressionType.typeID => DictionaryCompressionType case _ => throw new UnsupportedOperationException("Compression Type " + i) } implicit def intToColumnType(i: Int): ColumnType[_, _] = i match { - case 0 => INT - case 1 => LONG - case 2 => FLOAT - case 3 => DOUBLE - case 4 => BOOLEAN - case 5 => BYTE - case 6 => SHORT - case 7 => VOID - case 8 => STRING - case 9 => TIMESTAMP - case 10 => BINARY - case 11 => GENERIC + case INT.typeID => INT + case LONG.typeID => LONG + case FLOAT.typeID => FLOAT + case DOUBLE.typeID => DOUBLE + case BOOLEAN.typeID => BOOLEAN + case BYTE.typeID => BYTE + case SHORT.typeID => SHORT + case VOID.typeID => VOID + case STRING.typeID => STRING + case TIMESTAMP.typeID => TIMESTAMP + case BINARY.typeID => BINARY + case GENERIC.typeID => GENERIC } } @@ -78,6 +88,7 @@ object ColumnIterator { import shark.memstore2.column.Implicits._ def newIterator(b: ByteBuffer): ColumnIterator = { + // The first 4 bytes in the buffer indicates the column type. val buffer = b.duplicate().order(ByteOrder.nativeOrder()) val columnType: ColumnType[_, _] = buffer.getInt() val v = columnType match { diff --git a/src/main/scala/shark/memstore2/column/ColumnType.scala b/src/main/scala/shark/memstore2/column/ColumnType.scala index 068efe42..8945e03b 100644 --- a/src/main/scala/shark/memstore2/column/ColumnType.scala +++ b/src/main/scala/shark/memstore2/column/ColumnType.scala @@ -28,27 +28,56 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io._ -abstract class ColumnType[T, V](val typeID: Int, val defaultSize: Int) { +/** + * @param typeID A unique ID representing the type. + * @param defaultSize Default size in bytes for one element of type T (e.g. Int = 4). + * @tparam T Scala data type for the column. + * @tparam V Writable data type for the column. + */ +sealed abstract class ColumnType[T, V](val typeID: Int, val defaultSize: Int) { + /** + * Extract a value out of the buffer. + */ def extract(currentPos: Int, buffer: ByteBuffer): T + /** + * Append the given value v of type T into the given ByteBuffer. + */ def append(v: T, buffer: ByteBuffer) + /** + * Return the Scala data representation of the given object, using an object inspector. + */ def get(o: Object, oi: ObjectInspector): T - def actualSize(v: T) = defaultSize + /** + * Return the size of the value. This is used to calculate the size of variable length types + * such as byte arrays and strings. + */ + def actualSize(v: T): Int = defaultSize + /** + * Extract a value out of the buffer, and put it in the writable object. This is used as an + * optimization to reduce the temporary objects created, since the writable object can be reused. + */ def extractInto(currentPos: Int, buffer: ByteBuffer, writable: V) + /** + * Create a new writable object corresponding to this type. + */ def newWritable(): V + /** + * Create a duplicated copy of the value. + */ def clone(v: T): T = v } object INT extends ColumnType[Int, IntWritable](0, 4) { - override def append(v: Int, buffer: ByteBuffer) = { + override def append(v: Int, buffer: ByteBuffer) { buffer.putInt(v) } @@ -60,7 +89,7 @@ object INT extends ColumnType[Int, IntWritable](0, 4) { oi.asInstanceOf[IntObjectInspector].get(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: IntWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: IntWritable) { writable.set(extract(currentPos, buffer)) } @@ -70,7 +99,7 @@ object INT extends ColumnType[Int, IntWritable](0, 4) { object LONG extends ColumnType[Long, LongWritable](1, 8) { - override def append(v: Long, buffer: ByteBuffer) = { + override def append(v: Long, buffer: ByteBuffer) { buffer.putLong(v) } @@ -82,17 +111,17 @@ object LONG extends ColumnType[Long, LongWritable](1, 8) { oi.asInstanceOf[LongObjectInspector].get(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: LongWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: LongWritable) { writable.set(extract(currentPos, buffer)) } - def newWritable() = new LongWritable + override def newWritable() = new LongWritable } object FLOAT extends ColumnType[Float, FloatWritable](2, 4) { - override def append(v: Float, buffer: ByteBuffer) = { + override def append(v: Float, buffer: ByteBuffer) { buffer.putFloat(v) } @@ -104,38 +133,39 @@ object FLOAT extends ColumnType[Float, FloatWritable](2, 4) { oi.asInstanceOf[FloatObjectInspector].get(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: FloatWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: FloatWritable) { writable.set(extract(currentPos, buffer)) } - def newWritable() = new FloatWritable + override def newWritable() = new FloatWritable } object DOUBLE extends ColumnType[Double, DoubleWritable](3, 8) { - override def append(v: Double, buffer: ByteBuffer) = { + override def append(v: Double, buffer: ByteBuffer) { buffer.putDouble(v) } override def extract(currentPos: Int, buffer: ByteBuffer) = { buffer.getDouble() } + override def get(o: Object, oi: ObjectInspector): Double = { oi.asInstanceOf[DoubleObjectInspector].get(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: DoubleWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: DoubleWritable) { writable.set(extract(currentPos, buffer)) } - def newWritable() = new DoubleWritable + override def newWritable() = new DoubleWritable } object BOOLEAN extends ColumnType[Boolean, BooleanWritable](4, 1) { - override def append(v: Boolean, buffer: ByteBuffer) = { + override def append(v: Boolean, buffer: ByteBuffer) { buffer.put(if (v) 1.toByte else 0.toByte) } @@ -147,17 +177,17 @@ object BOOLEAN extends ColumnType[Boolean, BooleanWritable](4, 1) { oi.asInstanceOf[BooleanObjectInspector].get(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: BooleanWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: BooleanWritable) { writable.set(extract(currentPos, buffer)) } - def newWritable() = new BooleanWritable + override def newWritable() = new BooleanWritable } object BYTE extends ColumnType[Byte, ByteWritable](5, 1) { - override def append(v: Byte, buffer: ByteBuffer) = { + override def append(v: Byte, buffer: ByteBuffer) { buffer.put(v) } @@ -168,17 +198,17 @@ object BYTE extends ColumnType[Byte, ByteWritable](5, 1) { oi.asInstanceOf[ByteObjectInspector].get(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ByteWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ByteWritable) { writable.set(extract(currentPos, buffer)) } - def newWritable() = new ByteWritable + override def newWritable() = new ByteWritable } object SHORT extends ColumnType[Short, ShortWritable](6, 2) { - override def append(v: Short, buffer: ByteBuffer) = { + override def append(v: Short, buffer: ByteBuffer) { buffer.putShort(v) } @@ -190,7 +220,7 @@ object SHORT extends ColumnType[Short, ShortWritable](6, 2) { oi.asInstanceOf[ShortObjectInspector].get(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ShortWritable) = { + def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ShortWritable) { writable.set(extract(currentPos, buffer)) } @@ -200,7 +230,7 @@ object SHORT extends ColumnType[Short, ShortWritable](6, 2) { object VOID extends ColumnType[Void, NullWritable](7, 0) { - override def append(v: Void, buffer: ByteBuffer) = {} + override def append(v: Void, buffer: ByteBuffer) {} override def extract(currentPos: Int, buffer: ByteBuffer) = { throw new UnsupportedOperationException() @@ -208,7 +238,7 @@ object VOID extends ColumnType[Void, NullWritable](7, 0) { override def get(o: Object, oi: ObjectInspector) = null - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: NullWritable) = {} + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: NullWritable) {} override def newWritable() = NullWritable.get } @@ -246,7 +276,7 @@ object STRING extends ColumnType[Text, Text](8, 8) { override def actualSize(v: Text) = v.getLength() + 4 - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: Text) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: Text) { val length = buffer.getInt() var b = _bytesFld.get(writable).asInstanceOf[Array[Byte]] if (b == null || b.length < length) { @@ -257,7 +287,8 @@ object STRING extends ColumnType[Text, Text](8, 8) { _lengthFld.set(writable, length) } - def newWritable() = new Text + override def newWritable() = new Text + override def clone(v: Text) = { val t = new Text() t.set(v) @@ -268,7 +299,7 @@ object STRING extends ColumnType[Text, Text](8, 8) { object TIMESTAMP extends ColumnType[Timestamp, TimestampWritable](9, 12) { - override def append(v: Timestamp, buffer: ByteBuffer) = { + override def append(v: Timestamp, buffer: ByteBuffer) { buffer.putLong(v.getTime()) buffer.putInt(v.getNanos()) } @@ -284,11 +315,11 @@ object TIMESTAMP extends ColumnType[Timestamp, TimestampWritable](9, 12) { oi.asInstanceOf[TimestampObjectInspector].getPrimitiveJavaObject(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: TimestampWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: TimestampWritable) { writable.set(extract(currentPos, buffer)) } - def newWritable() = new TimestampWritable + override def newWritable() = new TimestampWritable } @@ -306,7 +337,7 @@ object BINARY extends ColumnType[BytesWritable, BytesWritable](10, 16) { f } - override def append(v: BytesWritable, buffer: ByteBuffer) = { + override def append(v: BytesWritable, buffer: ByteBuffer) { val length = v.getLength() buffer.putInt(length) buffer.put(v.getBytes(), 0, length) @@ -324,7 +355,7 @@ object BINARY extends ColumnType[BytesWritable, BytesWritable](10, 16) { } } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: BytesWritable) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: BytesWritable) { val length = buffer.getInt() var b = _bytesFld.get(writable).asInstanceOf[Array[Byte]] if (b == null || b.length < length) { @@ -335,7 +366,7 @@ object BINARY extends ColumnType[BytesWritable, BytesWritable](10, 16) { _lengthFld.set(writable, length) } - def newWritable() = new BytesWritable + override def newWritable() = new BytesWritable override def actualSize(v: BytesWritable) = v.getLength() + 4 } @@ -357,12 +388,12 @@ object GENERIC extends ColumnType[ByteStream.Output, ByteArrayRef](11, 16) { o.asInstanceOf[ByteStream.Output] } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ByteArrayRef) = { + override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ByteArrayRef) { val length = buffer.getInt() val a = new Array[Byte](length) buffer.get(a, 0, length) writable.setData(a) } - def newWritable() = new ByteArrayRef + override def newWritable() = new ByteArrayRef } diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index 4315b2e6..b20c0a98 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -11,9 +11,8 @@ import shark.memstore2.column.Implicits._ * The first element of the buffer at the point of initialization * is expected to be the type of compression indicator. */ -trait CompressedColumnIterator extends ColumnIterator{ +trait CompressedColumnIterator extends ColumnIterator { - private var _compressionType: CompressionType = _ private var _decoder: Iterator[_] = _ private var _current: Any = _ @@ -22,8 +21,8 @@ trait CompressedColumnIterator extends ColumnIterator{ def columnType: ColumnType[_,_] override def init() { - _compressionType = buffer.getInt() - _decoder = _compressionType match { + val compressionType: CompressionType = buffer.getInt() + _decoder = compressionType match { case DefaultCompressionType => new DefaultDecoder(buffer, columnType) case RLECompressionType => new RLDecoder(buffer, columnType) case DictionaryCompressionType => new DictDecoder(buffer, columnType) @@ -36,7 +35,7 @@ trait CompressedColumnIterator extends ColumnIterator{ _current = _decoder.next() } } - + override def hasNext = _decoder.hasNext override def current = _current.asInstanceOf[Object] @@ -81,7 +80,10 @@ class RLDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends Ite } } -class DictDecoder[V] (buffer:ByteBuffer, columnType: ColumnType[_, V]) extends Iterator[V] { +/** + * Dictionary encoding compression. + */ +class DictDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends Iterator[V] { private val _dictionary: Map[Short, V] = { val size = buffer.getInt() @@ -89,23 +91,23 @@ class DictDecoder[V] (buffer:ByteBuffer, columnType: ColumnType[_, V]) extends I var count = 0 while (count < size) { //read text, followed by index - val text = columnType.newWritable + val text = columnType.newWritable() columnType.extractInto(buffer.position(), buffer, text) val index = buffer.getShort() d.put(index, text.asInstanceOf[V]) - count+= 1 + count += 1 } d } override def hasNext = buffer.hasRemaining() - + override def next(): V = { val index = buffer.getShort() _dictionary.get(index) match { case Some(s) => s.asInstanceOf[V] - case None => throw new Exception("Bad index into Dictionary") + case None => throw new MemoryStoreException("Invalid dictionary index " + index) } } } diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index 03fad650..34214be9 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -1,7 +1,7 @@ package shark.memstore2.column -import java.nio.ByteBuffer -import java.nio.ByteOrder +import java.nio.{ByteBuffer, ByteOrder} + import scala.annotation.tailrec import scala.collection.mutable.HashMap @@ -12,8 +12,14 @@ trait CompressionAlgorithm { def compressionType: CompressionType + /** + * Tests whether the compression algorithm supports a specific column type. + */ def supportsType(t: ColumnType[_, _]): Boolean + /** + * Collect a value so we can update the compression ratio for this compression algorithm. + */ def gatherStatsForCompressibility[T](v: T, t: ColumnType[T, _]) /** @@ -21,6 +27,9 @@ trait CompressionAlgorithm { */ def compressionRatio: Double + /** + * Compress the given buffer and return the compressed data as a new buffer. + */ def compress[T](b: ByteBuffer, t: ColumnType[T, _]): ByteBuffer } @@ -30,8 +39,8 @@ case class CompressionType(typeID: Int) object DefaultCompressionType extends CompressionType(-1) object RLECompressionType extends CompressionType(0) + object DictionaryCompressionType extends CompressionType(1) -object RLEVariantCompressionType extends CompressionType(2) class NoCompression extends CompressionAlgorithm { @@ -39,7 +48,7 @@ class NoCompression extends CompressionAlgorithm { override def supportsType(t: ColumnType[_,_]) = true - override def gatherStatsForCompressibility[T](v: T, t: ColumnType[T,_]) = {} + override def gatherStatsForCompressibility[T](v: T, t: ColumnType[T,_]) {} override def compressionRatio: Double = 1.0 @@ -57,13 +66,16 @@ class NoCompression extends CompressionAlgorithm { } /** - * Implements Run Length Encoding + * Run-length encoding for columns with a lot of repeated values. */ class RLE extends CompressionAlgorithm { - private var _total: Int = 0 + private var _uncompressedSize: Int = 0 + private var _compressedSize: Int = 0 + + // Previous element, used to track how many runs and the run lengths. private var _prev: Any = _ + // Current run length. private var _run: Int = 0 - private var _size: Int = 0 override def compressionType = RLECompressionType @@ -74,35 +86,37 @@ class RLE extends CompressionAlgorithm { } } - override def gatherStatsForCompressibility[T](v: T, t: ColumnType[T,_]) = { + override def gatherStatsForCompressibility[T](v: T, t: ColumnType[T,_]) { val s = t.actualSize(v) if (_prev == null) { + // This is the very first run. _prev = t.clone(v) _run = 1 } else { if (_prev.equals(v)) { + // Add one to the current run's length. _run += 1 } else { - // flush run into size - _size += (t.actualSize(_prev.asInstanceOf[T]) + 4) + // Start a new run. Update the current run length. + _compressedSize += (t.actualSize(_prev.asInstanceOf[T]) + 4) _prev = t.clone(v) _run = 1 } } - _total += s + _uncompressedSize += s } // Note that we don't actually track the size of the last run into account to simplify the // logic a little bit. - override def compressionRatio = _size / (_total + 0.0) + override def compressionRatio = _compressedSize / (_uncompressedSize + 0.0) - override def compress[T](b: ByteBuffer, t: ColumnType[T,_]) = { + override def compress[T](b: ByteBuffer, t: ColumnType[T,_]): ByteBuffer = { // Add the size of the last run to the _size if (_prev != null) { - _size += t.actualSize(_prev.asInstanceOf[T]) + 4 + _compressedSize += t.actualSize(_prev.asInstanceOf[T]) + 4 } - val compressedBuffer = ByteBuffer.allocate(_size + 4 + 4) + val compressedBuffer = ByteBuffer.allocate(_compressedSize + 4 + 4) compressedBuffer.order(ByteOrder.nativeOrder()) compressedBuffer.putInt(b.getInt()) compressedBuffer.putInt(compressionType.typeID) @@ -112,7 +126,7 @@ class RLE extends CompressionAlgorithm { } @tailrec private final def encode[T](currentBuffer: ByteBuffer, - compressedBuffer: ByteBuffer, currentRun: (T, Int), t: ColumnType[T,_]) { + compressedBuffer: ByteBuffer, currentRun: (T, Int), t: ColumnType[T,_]) { def writeOutRun() { t.append(currentRun._1, compressedBuffer) compressedBuffer.putInt(currentRun._2) @@ -137,93 +151,103 @@ class RLE extends CompressionAlgorithm { } } +/** + * Dictionary encoding for columns with small cardinality. + */ class DictionaryEncoding extends CompressionAlgorithm { private val MAX_DICT_SIZE = Short.MaxValue - 1 // 32K unique values allowed - private val _dictionary = new HashMap[Any, Short]() - private val indexSize = 2 // Short + private var _dictionary = new HashMap[Any, Short]() + + // We use a short integer to store the dictionary index, which takes 2 bytes. + private val indexSize = 2 + + // Size of the dictionary, in bytes. Initialize the dictionary size to 4 since we use an int + // to store the number of elements in the dictionary. + private var _dictionarySize = 4 - private var _dictionarySize = 0 - private var _totalSize = 0 + // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary + // overflows. + private var _uncompressedSize = 0 + + // Total number of elements. private var _count = 0 - private var _index: Short = 0 + + // If the number of distinct elements is too large, we discard the use of dictionary + // encoding and set the overflow flag to true. private var _overflow = false override def compressionType = DictionaryCompressionType override def supportsType(t: ColumnType[_, _]) = t match { - case STRING => true - case LONG => true - case INT => true + case STRING | LONG | INT => true case _ => false } - private def encode[T](v: T, t: ColumnType[T, _], sizeFunc:T => Int): Short = { - _count += 1 - val size = sizeFunc(v) - _totalSize += size - if (_dictionary.size < MAX_DICT_SIZE) { - val s = t.clone(v) - _dictionary.get(s) match { - case Some(index) => index - case None => { - _dictionary.put(s, _index) - _index = (1 + _index).toShort - _dictionarySize += (size + indexSize) - _index + override def gatherStatsForCompressibility[T](v: T, t: ColumnType[T, _]) { + // Use this function to build up a dictionary. + if (!_overflow) { + val size = t.actualSize(v) + _count += 1 + _uncompressedSize += size + + if (!_dictionary.contains(v)) { + // The dictionary doesn't contain the value. Add the value to the dictionary if we haven't + // overflown yet. + if (_dictionary.size < MAX_DICT_SIZE) { + _dictionary.put(t.clone(v), _dictionary.size.toShort) + _dictionarySize += size + indexSize + } else { + // Overflown. Release the dictionary immediately to lower memory pressure. + _overflow = true + _dictionary = null } } - } else { - _overflow = true - -1 } } - override def gatherStatsForCompressibility[T](v: T, t: ColumnType[T, _]) = { - //need an estimate of the # of uniques so we can build an appropriate - //dictionary if needed. More precisely, we only need a lower bound - //on # of uniques. - val size = t.actualSize(v) - encode(v, t, { _:T => size}) - } - /** - * return score between 0 and 1, smaller score imply higher compressibility. - * return Double.MaxValue to indicate overflow so that dict does not get chosen. + * Return the compression ratio if encoded with dictionary encoding. If the dictionary + * cardinality (i.e. the number of distinct elements) is bigger than 32K, we return an + * arbitrary number greater than 1.0. */ - override def compressionRatio: Double = { - if (_overflow) Double.MaxValue else (bufferSize) / (_totalSize + 0.0) + override def compressionRatio: Double = compressedSize / (_uncompressedSize + 0.0) + + private def compressedSize: Int = { + // Total compressed size = + // size of the dictionary + + // the number of elements * dictionary encoded size (short) + + // an integer for compression type + // an integer for column type + if (_overflow) Int.MaxValue else _dictionarySize + _count * indexSize + 4 + 4 } - private def writeDictionary[T](compressedBuffer: ByteBuffer, t: ColumnType[T, _]) { - //store dictionary size + override def compress[T](b: ByteBuffer, t: ColumnType[T, _]): ByteBuffer = { + if (_overflow) { + throw new MemoryStoreException( + "Dictionary encoding should not be used because we have overflown the dictionary.") + } + + // Create a new buffer and store the compression type and column type. + val compressedBuffer = ByteBuffer.allocate(compressedSize) + compressedBuffer.order(ByteOrder.nativeOrder()) + compressedBuffer.putInt(b.getInt()) + compressedBuffer.putInt(compressionType.typeID) + + // Write out the dictionary. compressedBuffer.putInt(_dictionary.size) - //store the dictionary _dictionary.foreach { x => t.append(x._1.asInstanceOf[T], compressedBuffer) compressedBuffer.putShort(x._2) } - } - - private def dictionarySize = _dictionarySize + 4 - private def bufferSize = { _count*indexSize + dictionarySize + 4 + 4 } - override def compress[T](b: ByteBuffer, t: ColumnType[T, _]): ByteBuffer = { - //build a dictionary of given size - val compressedBuffer = ByteBuffer.allocate(bufferSize) - compressedBuffer.order(ByteOrder.nativeOrder()) - compressedBuffer.putInt(b.getInt()) - compressedBuffer.putInt(compressionType.typeID) - //store dictionary size - writeDictionary(compressedBuffer, t) - //traverse the original buffer + // Write out the encoded values, each is represented by a short integer. while (b.hasRemaining()) { val v = t.extract(b.position(), b) - _dictionary.get(v).map { index => - compressedBuffer.putShort(index) - } - + compressedBuffer.putShort(_dictionary(v)) } + + // Rewind the compressed buffer and return it. compressedBuffer.rewind() compressedBuffer } diff --git a/src/main/scala/shark/memstore2/column/MemoryStoreException.scala b/src/main/scala/shark/memstore2/column/MemoryStoreException.scala new file mode 100644 index 00000000..5db2631d --- /dev/null +++ b/src/main/scala/shark/memstore2/column/MemoryStoreException.scala @@ -0,0 +1,21 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column + + +class MemoryStoreException(message: String) extends Exception(message) diff --git a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala index 3ab34a91..bd9ee77e 100644 --- a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala @@ -15,7 +15,7 @@ class NullableColumnIterator(delegate: ColumnIterator, buffer: ByteBuffer) exten private var _nulls = 0 private var _isNull = false - private var _currentNullIndex:Int = _ + private var _currentNullIndex: Int = _ private var _pos = 0 override def init() { @@ -42,15 +42,7 @@ class NullableColumnIterator(delegate: ColumnIterator, buffer: ByteBuffer) exten _pos += 1 } - override def hasNext: Boolean = { - if (_nulls < _nullCount) { - true - } else { - delegate.hasNext - } - } + override def hasNext: Boolean = (_nulls < _nullCount) || delegate.hasNext - def current: Object = { - if (_isNull) null else delegate.current - } + def current: Object = if (_isNull) null else delegate.current } diff --git a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala index 12a66f98..d74541d9 100644 --- a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala @@ -12,117 +12,110 @@ import shark.memstore2.column.Implicits._ class CompressedColumnIteratorSuite extends FunSuite { - val booleanList = Array[Boolean](true, true, false, true) - val byteList = Array[Byte](10, 10, 20, 10) - val shortList = byteList.map { i => (Short.MaxValue - i).toShort } - val iList = byteList.map { i => Int.MaxValue - i.toInt } - val lList = iList.map { i => Long.MaxValue - i.toLong } - val sList = iList.map { i => new Text(i.toString) } - - /** Generic tester across types and encodings - * - */ + /** + * Generic tester across types and encodings + */ def testList[T, W]( - l: Seq[T], - u: ColumnType[T, _], - algo: CompressionAlgorithm, - compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b, - shouldNotCompress: Boolean = false) { - - val b = ByteBuffer.allocate(1024 + (3*40*l.size)) + l: Seq[T], + t: ColumnType[T, _], + algo: CompressionAlgorithm, + compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b, + shouldNotCompress: Boolean = false) + { + val b = ByteBuffer.allocate(1024 + (3 * 40 * l.size)) b.order(ByteOrder.nativeOrder()) - b.putInt(u.typeID) + b.putInt(t.typeID) l.foreach { item => - u.append(item.asInstanceOf[T], b) - algo.gatherStatsForCompressibility(item, u.asInstanceOf[ColumnType[Any, _]]) + t.append(item, b) + algo.gatherStatsForCompressibility(item, t.asInstanceOf[ColumnType[Any, _]]) } b.limit(b.position()) b.rewind() - val compressedBuffer = algo.compress(b, u) + if (shouldNotCompress) { assert(algo.compressionRatio >= 1.0) - info("CompressionRatio " + algo.compressionRatio) } else { - + val compressedBuffer = algo.compress(b, t) val iter = new TestIterator(compressedBuffer, compressedBuffer.getInt()) - val oi: ObjectInspector = u match { + val oi: ObjectInspector = t match { case BOOLEAN => PrimitiveObjectInspectorFactory.writableBooleanObjectInspector case BYTE => PrimitiveObjectInspectorFactory.writableByteObjectInspector case SHORT => PrimitiveObjectInspectorFactory.writableShortObjectInspector case INT => PrimitiveObjectInspectorFactory.writableIntObjectInspector case LONG => PrimitiveObjectInspectorFactory.writableLongObjectInspector case STRING => PrimitiveObjectInspectorFactory.writableStringObjectInspector + case _ => throw new UnsupportedOperationException("Unsupported compression type " + t) } l.foreach { x => iter.next() - assert(compareFunc(u.get(iter.current, oi), x)) - // assert(u.get(iter.current, oi) === x) + assert(compareFunc(t.get(iter.current, oi), x)) } - assert(false === iter.hasNext) // no extras at the end + + // Make sure we reach the end of the iterator. + assert(!iter.hasNext) } } - test("RLE Decompression Boolean") { - testList(booleanList, BOOLEAN, new RLE()) + test("RLE Boolean") { + testList(Seq[Boolean](true, true, false, true), BOOLEAN, new RLE()) } - test("RLE Decompression Byte") { - testList(byteList, BYTE, new RLE()) + test("RLE Byte") { + testList(Seq[Byte](10, 10, 20, 10), BYTE, new RLE()) } - test("RLE Decompression Short") { - testList(shortList, SHORT, new RLE()) + test("RLE Short") { + testList(Seq[Short](10, 10, 10, 20000, 20000, 20000, 500, 500, 500, 500), SHORT, new RLE()) } - test("RLE Decompression Int") { - testList(iList, INT, new RLE()) + test("RLE Int") { + testList(Seq[Int](1000000, 1000000, 1000000, 1000000, 900000, 99), INT, new RLE()) } - test("RLE Decompression Long") { - testList(lList, LONG, new RLE()) + test("RLE Long") { + val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) + testList(longs, LONG, new RLE()) } - test("RLE Decompression String") { - testList(sList, STRING, new RLE(), (a: Text, b: Text) => a.hashCode == b.hashCode) + test("RLE String") { + val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "!", "!").map(s => new Text(s)) + testList(strs, STRING, new RLE(), (a: Text, b: Text) => a.equals(b)) } - test("Dictionary Decompression Int") { - testList(iList, INT, new DictionaryEncoding()) + test("Dictionary Encoded Int") { + testList(Seq[Int](1000000, 1000000, 1000000, 1000000, 900000, 99), INT, new DictionaryEncoding) } - test("Dictionary Decompression Long") { - testList(lList, LONG, new DictionaryEncoding()) + test("Dictionary Encoded Long") { + val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) + testList(longs, LONG, new DictionaryEncoding) } - test("Dictionary Decompression String") { - testList(sList, STRING, new DictionaryEncoding(), (a: Text, b: Text) => a.hashCode == b.hashCode) + test("Dictionary Encoded String") { + val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "!", "!").map(s => new Text(s)) + testList(strs, STRING, new DictionaryEncoding, (a: Text, b: Text) => a.equals(b), + shouldNotCompress = false) } - test("Dictionary Decompression at limit of unique values") { - val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } - val iiList = byteList.map { i => i.toInt } - val hugeList = List.concat(iiList, alternating, iiList) - assert(hugeList.size === (8 + 2*(Short.MaxValue-1))) - testList(hugeList, INT, new DictionaryEncoding()) + test("Dictionary Encoding at limit of unique values") { + val ints = Range(0, Short.MaxValue - 1).flatMap(i => Iterator(i, i, i)) + testList(ints, INT, new DictionaryEncoding) } - test("Dictionary Decompression - should not compress") { - val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } - val hugeList = List.concat(iList, alternating, iList) - assert(hugeList.size === (8 + 2*(Short.MaxValue-1))) - testList(hugeList, INT, new DictionaryEncoding(), (a: Int, b: Int) => a == b, true) + test("Dictionary Encoding - should not compress") { + val ints = Range(0, Short.MaxValue.toInt) + testList(ints, INT, new DictionaryEncoding, (a: Int, b: Int) => a == b, + shouldNotCompress = true) } test("RLE - should not compress") { - val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } - val hugeList = List.concat(iList, alternating, iList) - assert(hugeList.size === (8 + 2*(Short.MaxValue-1))) - testList(hugeList, INT, new RLE(), (a: Int, b: Int) => a == b, true) + val ints = Range(0, Short.MaxValue.toInt + 1) + testList(ints, INT, new RLE, (a: Int, b: Int) => a == b, shouldNotCompress = true) } - } - class TestIterator(val buffer: ByteBuffer, val columnType: ColumnType[_,_]) - extends CompressedColumnIterator + +class TestIterator(val buffer: ByteBuffer, val columnType: ColumnType[_,_]) + extends CompressedColumnIterator diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index 00218547..d006c61d 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -1,23 +1,28 @@ package shark.memstore2.column -import java.nio.ByteBuffer -import java.nio.ByteOrder +import java.nio.{ByteBuffer, ByteOrder} + import scala.collection.mutable.HashMap -import org.scalatest.FunSuite import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.io.Text +import org.scalatest.FunSuite + import shark.memstore2.column.ColumnStats._ class CompressionAlgorithmSuite extends FunSuite { + // TODO: clean these tests. + test("Compressed Column Builder") { + class TestColumnBuilder(val stats: ColumnStats[Int], val t: ColumnType[Int,_]) - extends CompressedColumnBuilder[Int] { + extends CompressedColumnBuilder[Int] { compressionSchemes = Seq(new RLE()) override def shouldApply(scheme: CompressionAlgorithm) = true } + val b = new TestColumnBuilder(new NoOpStats, INT) b.initialize(100) val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector @@ -31,7 +36,6 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 123) assert(compressedBuffer.getInt() == 2) - } test("RLE Strings") { @@ -39,10 +43,7 @@ class CompressionAlgorithmSuite extends FunSuite { b.order(ByteOrder.nativeOrder()) b.putInt(STRING.typeID) val rle = new RLE() - Array[Text](new Text("abc"), - new Text("abc"), - new Text("efg"), - new Text("abc")).foreach { text => + Seq[Text](new Text("abc"), new Text("abc"), new Text("efg"), new Text("abc")).foreach { text => STRING.append(text, b) rle.gatherStatsForCompressibility(text, STRING) } @@ -51,9 +52,9 @@ class CompressionAlgorithmSuite extends FunSuite { val compressedBuffer = rle.compress(b, STRING) assert(compressedBuffer.getInt() == STRING.typeID) assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals (new Text("abc"))) + assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals(new Text("abc"))) assert(compressedBuffer.getInt() == 2) - assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals (new Text("efg"))) + assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals(new Text("efg"))) assert(compressedBuffer.getInt() == 1) } @@ -135,7 +136,7 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 20) assert(compressedBuffer.getInt() == 400) } - + test("RLE perf") { val b = ByteBuffer.allocate(4000008) b.order(ByteOrder.nativeOrder()) @@ -169,7 +170,7 @@ class CompressionAlgorithmSuite extends FunSuite { b.putInt(u.typeID) val de = new DictionaryEncoding() l.foreach { item => - u.append(item.asInstanceOf[T], b) + u.append(item, b) de.gatherStatsForCompressibility(item, u.asInstanceOf[ColumnType[Any, _]]) } b.limit(b.position()) diff --git a/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala b/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala index d2c6a0dc..86048d10 100644 --- a/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala +++ b/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala @@ -22,7 +22,7 @@ class NullableColumnBuilderSuite extends FunSuite { } } - test("Grow") { + test("Buffer size auto growth") { val c = new StringColumnBuilder() c.initialize(4) val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector @@ -108,7 +108,7 @@ class NullableColumnBuilderSuite extends FunSuite { val c = new IntColumnBuilder() c.initialize(4) val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector - Range(1,1000).foreach { x => + Range(1, 1000).foreach { x => c.append(x.asInstanceOf[Object], oi) } val b = c.build() From 000d26b7f5e5db7e25bef63f20e5e98c344093fb Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 6 Sep 2013 16:58:08 +0800 Subject: [PATCH 013/331] Simplified dictionary encoding even more. 1. No need to store the "index" for each value in the dictionary. 2. Use an array instead of a hash map for decoding. --- .../shark/memstore2/column/ColumnType.scala | 14 ++++++- .../column/CompressedColumnIterator.scala | 26 +++++-------- .../column/CompressionAlgorithm.scala | 25 +++++++++---- .../CompressedColumnIteratorSuite.scala | 8 +++- .../column/CompressionAlgorithmSuite.scala | 37 ++++++++++++------- 5 files changed, 71 insertions(+), 39 deletions(-) diff --git a/src/main/scala/shark/memstore2/column/ColumnType.scala b/src/main/scala/shark/memstore2/column/ColumnType.scala index 8945e03b..026ed57b 100644 --- a/src/main/scala/shark/memstore2/column/ColumnType.scala +++ b/src/main/scala/shark/memstore2/column/ColumnType.scala @@ -34,7 +34,19 @@ import org.apache.hadoop.io._ * @tparam T Scala data type for the column. * @tparam V Writable data type for the column. */ -sealed abstract class ColumnType[T, V](val typeID: Int, val defaultSize: Int) { +sealed abstract class ColumnType[T : ClassManifest, V : ClassManifest]( + val typeID: Int, val defaultSize: Int) { + + /** + * Scala class manifest. Can be used to create primitive arrays and hash tables. + */ + def scalaManifest: ClassManifest[T] = classManifest[T] + + /** + * Scala class manifest for the writable type. Can be used to create primitive arrays and + * hash tables. + */ + def writableManifest: ClassManifest[V] = classManifest[V] /** * Extract a value out of the buffer. diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index b20c0a98..4870d85e 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -2,8 +2,6 @@ package shark.memstore2.column import java.nio.ByteBuffer -import scala.collection.mutable.{Map, HashMap} - import shark.memstore2.column.Implicits._ /** @@ -85,29 +83,25 @@ class RLDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends Ite */ class DictDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends Iterator[V] { - private val _dictionary: Map[Short, V] = { + // Dictionary in the form of an array. The index is the encoded value, and the value is the + // decompressed value. + private val _dictionary: Array[V] = { val size = buffer.getInt() - val d = new HashMap[Short, V]() + val arr = columnType.writableManifest.newArray(size) var count = 0 while (count < size) { - //read text, followed by index - val text = columnType.newWritable() - columnType.extractInto(buffer.position(), buffer, text) - val index = buffer.getShort() - d.put(index, text.asInstanceOf[V]) - + val writable = columnType.newWritable() + columnType.extractInto(buffer.position(), buffer, writable) + arr(count) = writable.asInstanceOf[V] count += 1 } - d + arr } override def hasNext = buffer.hasRemaining() override def next(): V = { - val index = buffer.getShort() - _dictionary.get(index) match { - case Some(s) => s.asInstanceOf[V] - case None => throw new MemoryStoreException("Invalid dictionary index " + index) - } + val index = buffer.getShort().toInt + _dictionary(index) } } diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index 34214be9..bd2f865f 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -3,7 +3,7 @@ package shark.memstore2.column import java.nio.{ByteBuffer, ByteOrder} import scala.annotation.tailrec -import scala.collection.mutable.HashMap +import scala.collection.mutable.{ArrayBuffer, HashMap} /** * API for Compression @@ -152,13 +152,20 @@ class RLE extends CompressionAlgorithm { } /** - * Dictionary encoding for columns with small cardinality. + * Dictionary encoding for columns with small cardinality. This algorithm encodes values into + * short integers (2 byte each). It can support up to 32k distinct values. */ class DictionaryEncoding extends CompressionAlgorithm { - private val MAX_DICT_SIZE = Short.MaxValue - 1 // 32K unique values allowed + // 32K unique values allowed + private val MAX_DICT_SIZE = Short.MaxValue - 1 + + // The dictionary that maps a value to the encoded short integer. private var _dictionary = new HashMap[Any, Short]() + // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself. + private var _values = new ArrayBuffer[Any](1024) + // We use a short integer to store the dictionary index, which takes 2 bytes. private val indexSize = 2 @@ -195,12 +202,15 @@ class DictionaryEncoding extends CompressionAlgorithm { // The dictionary doesn't contain the value. Add the value to the dictionary if we haven't // overflown yet. if (_dictionary.size < MAX_DICT_SIZE) { - _dictionary.put(t.clone(v), _dictionary.size.toShort) - _dictionarySize += size + indexSize + val clone = t.clone(v) + _values.append(clone) + _dictionary.put(clone, _dictionary.size.toShort) + _dictionarySize += size } else { // Overflown. Release the dictionary immediately to lower memory pressure. _overflow = true _dictionary = null + _values = null } } } @@ -236,9 +246,8 @@ class DictionaryEncoding extends CompressionAlgorithm { // Write out the dictionary. compressedBuffer.putInt(_dictionary.size) - _dictionary.foreach { x => - t.append(x._1.asInstanceOf[T], compressedBuffer) - compressedBuffer.putShort(x._2) + _values.foreach { v => + t.append(v.asInstanceOf[T], compressedBuffer) } // Write out the encoded values, each is represented by a short integer. diff --git a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala index d74541d9..316e7cc8 100644 --- a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala @@ -13,7 +13,13 @@ import shark.memstore2.column.Implicits._ class CompressedColumnIteratorSuite extends FunSuite { /** - * Generic tester across types and encodings + * Generic tester across types and encodings. The function applies the given compression + * algorithm on the given sequence of values, and test whether the resulting iterator gives + * the same sequence of values. + * + * If we expect the compression algorithm to not compress the data, we should set the + * shouldNotCompress flag to true. This way, it doesn't actually create a compressed buffer, + * but simply tests the compression ratio returned by the algorithm is >= 1.0. */ def testList[T, W]( l: Seq[T], diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index d006c61d..7d7d3ec7 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -15,7 +15,7 @@ class CompressionAlgorithmSuite extends FunSuite { // TODO: clean these tests. - test("Compressed Column Builder") { + test("CompressedColumnBuilder using RLE") { class TestColumnBuilder(val stats: ColumnStats[Int], val t: ColumnType[Int,_]) extends CompressedColumnBuilder[Int] { @@ -36,6 +36,9 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 123) assert(compressedBuffer.getInt() == 2) + assert(compressedBuffer.getInt() == 56) + assert(compressedBuffer.getInt() == 2) + assert(!compressedBuffer.hasRemaining) } test("RLE Strings") { @@ -56,9 +59,12 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 2) assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals(new Text("efg"))) assert(compressedBuffer.getInt() == 1) + assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals(new Text("abc"))) + assert(compressedBuffer.getInt() == 1) + assert(!compressedBuffer.hasRemaining) } - test("RLE no encoding") { + test("RLE int with run length 1") { val b = ByteBuffer.allocate(16) b.order(ByteOrder.nativeOrder()) b.putInt(INT.typeID) @@ -76,14 +82,15 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 1) assert(compressedBuffer.getInt() == 56) assert(compressedBuffer.getInt() == 1) + assert(!compressedBuffer.hasRemaining) } - test("RLE perfect encoding Int") { + test("RLE int single run") { val b = ByteBuffer.allocate(4008) b.order(ByteOrder.nativeOrder()) b.putInt(INT.typeID) val rle = new RLE() - Range(0,1000).foreach { x => + Range(0, 1000).foreach { x => b.putInt(6) rle.gatherStatsForCompressibility(6, INT) } @@ -94,14 +101,15 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == RLECompressionType.typeID) assert(compressedBuffer.getInt() == 6) assert(compressedBuffer.getInt() == 1000) + assert(!compressedBuffer.hasRemaining) } - test("RLE perfect encoding Long") { + test("RLE long single run") { val b = ByteBuffer.allocate(8008) b.order(ByteOrder.nativeOrder()) b.putInt(LONG.typeID) val rle = new RLE() - Range(0,1000).foreach { x => + Range(0, 1000).foreach { x => b.putLong(Long.MaxValue - 6) rle.gatherStatsForCompressibility(Long.MaxValue - 6, LONG) } @@ -112,16 +120,17 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == RLECompressionType.typeID) assert(compressedBuffer.getLong() == Long.MaxValue - 6) assert(compressedBuffer.getInt() == 1000) + assert(!compressedBuffer.hasRemaining) } - test("RLE mixture") { + test("RLE int 3 runs") { val b = ByteBuffer.allocate(4008) b.order(ByteOrder.nativeOrder()) b.putInt(INT.typeID) val items = Array[Int](10, 20, 40) val rle = new RLE() - Range(0,1000).foreach { x => + Range(0, 1000).foreach { x => val v = if (x < 100) items(0) else if (x < 500) items(1) else items(2) b.putInt(v) rle.gatherStatsForCompressibility(v, INT) @@ -135,26 +144,29 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 100) assert(compressedBuffer.getInt() == 20) assert(compressedBuffer.getInt() == 400) + assert(compressedBuffer.getInt() == 40) + assert(compressedBuffer.getInt() == 500) + assert(!compressedBuffer.hasRemaining) } - test("RLE perf") { + test("RLE int single long run") { val b = ByteBuffer.allocate(4000008) b.order(ByteOrder.nativeOrder()) b.putInt(INT.typeID) val rle = new RLE() - Range(0,1000000).foreach { x => + Range(0, 1000000).foreach { x => b.putInt(6) rle.gatherStatsForCompressibility(6, INT) } b.limit(b.position()) b.rewind() val compressedBuffer = rle.compress(b, INT) - //first 4 bytes is the compression scheme assert(compressedBuffer.getInt() == RLECompressionType.typeID) assert(compressedBuffer.getInt() == INT.typeID) assert(compressedBuffer.getInt() == 6) assert(compressedBuffer.getInt() == 1000000) + assert(!compressedBuffer.hasRemaining) } test("Dictionary Encoding") { @@ -183,8 +195,7 @@ class CompressionAlgorithmSuite extends FunSuite { var count = 0 while (count < expectedDictSize) { val v = u.extract(compressedBuffer.position(), compressedBuffer) - val index = compressedBuffer.getShort() - dictionary.put(index, u.clone(v)) + dictionary.put(dictionary.size.toShort, u.clone(v)) count += 1 } assert(dictionary.get(0).get.equals(l(0))) From 16d9d89d45275e2f2a3d94038b75a4568e71aa8b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 6 Sep 2013 17:08:38 +0800 Subject: [PATCH 014/331] Removed the position argument in ColumnType.extract / extractInto. --- .../shark/memstore2/column/ColumnType.scala | 77 ++++++++++--------- .../column/CompressedColumnIterator.scala | 6 +- .../column/CompressionAlgorithm.scala | 4 +- .../memstore2/column/ColumnTypeSuite.scala | 12 +-- .../column/CompressionAlgorithmSuite.scala | 8 +- 5 files changed, 54 insertions(+), 53 deletions(-) diff --git a/src/main/scala/shark/memstore2/column/ColumnType.scala b/src/main/scala/shark/memstore2/column/ColumnType.scala index 026ed57b..3d50d2e6 100644 --- a/src/main/scala/shark/memstore2/column/ColumnType.scala +++ b/src/main/scala/shark/memstore2/column/ColumnType.scala @@ -49,9 +49,9 @@ sealed abstract class ColumnType[T : ClassManifest, V : ClassManifest]( def writableManifest: ClassManifest[V] = classManifest[V] /** - * Extract a value out of the buffer. + * Extract a value out of the buffer at the buffer's current position. */ - def extract(currentPos: Int, buffer: ByteBuffer): T + def extract(buffer: ByteBuffer): T /** * Append the given value v of type T into the given ByteBuffer. @@ -70,10 +70,11 @@ sealed abstract class ColumnType[T : ClassManifest, V : ClassManifest]( def actualSize(v: T): Int = defaultSize /** - * Extract a value out of the buffer, and put it in the writable object. This is used as an - * optimization to reduce the temporary objects created, since the writable object can be reused. + * Extract a value out of the buffer at the buffer's current position, and put it in the writable + * object. This is used as an optimization to reduce the temporary objects created, since the + * writable object can be reused. */ - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: V) + def extractInto(buffer: ByteBuffer, writable: V) /** * Create a new writable object corresponding to this type. @@ -93,7 +94,7 @@ object INT extends ColumnType[Int, IntWritable](0, 4) { buffer.putInt(v) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { buffer.getInt() } @@ -101,8 +102,8 @@ object INT extends ColumnType[Int, IntWritable](0, 4) { oi.asInstanceOf[IntObjectInspector].get(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: IntWritable) { - writable.set(extract(currentPos, buffer)) + override def extractInto(buffer: ByteBuffer, writable: IntWritable) { + writable.set(extract(buffer)) } override def newWritable() = new IntWritable @@ -115,7 +116,7 @@ object LONG extends ColumnType[Long, LongWritable](1, 8) { buffer.putLong(v) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { buffer.getLong() } @@ -123,8 +124,8 @@ object LONG extends ColumnType[Long, LongWritable](1, 8) { oi.asInstanceOf[LongObjectInspector].get(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: LongWritable) { - writable.set(extract(currentPos, buffer)) + override def extractInto(buffer: ByteBuffer, writable: LongWritable) { + writable.set(extract(buffer)) } override def newWritable() = new LongWritable @@ -137,7 +138,7 @@ object FLOAT extends ColumnType[Float, FloatWritable](2, 4) { buffer.putFloat(v) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { buffer.getFloat() } @@ -145,8 +146,8 @@ object FLOAT extends ColumnType[Float, FloatWritable](2, 4) { oi.asInstanceOf[FloatObjectInspector].get(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: FloatWritable) { - writable.set(extract(currentPos, buffer)) + override def extractInto(buffer: ByteBuffer, writable: FloatWritable) { + writable.set(extract(buffer)) } override def newWritable() = new FloatWritable @@ -159,7 +160,7 @@ object DOUBLE extends ColumnType[Double, DoubleWritable](3, 8) { buffer.putDouble(v) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { buffer.getDouble() } @@ -167,8 +168,8 @@ object DOUBLE extends ColumnType[Double, DoubleWritable](3, 8) { oi.asInstanceOf[DoubleObjectInspector].get(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: DoubleWritable) { - writable.set(extract(currentPos, buffer)) + override def extractInto(buffer: ByteBuffer, writable: DoubleWritable) { + writable.set(extract(buffer)) } override def newWritable() = new DoubleWritable @@ -181,7 +182,7 @@ object BOOLEAN extends ColumnType[Boolean, BooleanWritable](4, 1) { buffer.put(if (v) 1.toByte else 0.toByte) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { if (buffer.get() == 1) true else false } @@ -189,8 +190,8 @@ object BOOLEAN extends ColumnType[Boolean, BooleanWritable](4, 1) { oi.asInstanceOf[BooleanObjectInspector].get(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: BooleanWritable) { - writable.set(extract(currentPos, buffer)) + override def extractInto(buffer: ByteBuffer, writable: BooleanWritable) { + writable.set(extract(buffer)) } override def newWritable() = new BooleanWritable @@ -203,15 +204,15 @@ object BYTE extends ColumnType[Byte, ByteWritable](5, 1) { buffer.put(v) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { buffer.get() } override def get(o: Object, oi: ObjectInspector): Byte = { oi.asInstanceOf[ByteObjectInspector].get(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ByteWritable) { - writable.set(extract(currentPos, buffer)) + override def extractInto(buffer: ByteBuffer, writable: ByteWritable) { + writable.set(extract(buffer)) } override def newWritable() = new ByteWritable @@ -224,7 +225,7 @@ object SHORT extends ColumnType[Short, ShortWritable](6, 2) { buffer.putShort(v) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { buffer.getShort() } @@ -232,8 +233,8 @@ object SHORT extends ColumnType[Short, ShortWritable](6, 2) { oi.asInstanceOf[ShortObjectInspector].get(o) } - def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ShortWritable) { - writable.set(extract(currentPos, buffer)) + def extractInto(buffer: ByteBuffer, writable: ShortWritable) { + writable.set(extract(buffer)) } def newWritable() = new ShortWritable @@ -244,13 +245,13 @@ object VOID extends ColumnType[Void, NullWritable](7, 0) { override def append(v: Void, buffer: ByteBuffer) {} - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { throw new UnsupportedOperationException() } override def get(o: Object, oi: ObjectInspector) = null - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: NullWritable) {} + override def extractInto(buffer: ByteBuffer, writable: NullWritable) {} override def newWritable() = NullWritable.get } @@ -276,9 +277,9 @@ object STRING extends ColumnType[Text, Text](8, 8) { buffer.put(v.getBytes(), 0, length) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { val t = new Text() - extractInto(currentPos, buffer, t) + extractInto(buffer, t) t } @@ -288,7 +289,7 @@ object STRING extends ColumnType[Text, Text](8, 8) { override def actualSize(v: Text) = v.getLength() + 4 - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: Text) { + override def extractInto(buffer: ByteBuffer, writable: Text) { val length = buffer.getInt() var b = _bytesFld.get(writable).asInstanceOf[Array[Byte]] if (b == null || b.length < length) { @@ -316,7 +317,7 @@ object TIMESTAMP extends ColumnType[Timestamp, TimestampWritable](9, 12) { buffer.putInt(v.getNanos()) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { val ts = new Timestamp(0) ts.setTime(buffer.getLong()) ts.setNanos(buffer.getInt()) @@ -327,8 +328,8 @@ object TIMESTAMP extends ColumnType[Timestamp, TimestampWritable](9, 12) { oi.asInstanceOf[TimestampObjectInspector].getPrimitiveJavaObject(o) } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: TimestampWritable) { - writable.set(extract(currentPos, buffer)) + override def extractInto(buffer: ByteBuffer, writable: TimestampWritable) { + writable.set(extract(buffer)) } override def newWritable() = new TimestampWritable @@ -355,7 +356,7 @@ object BINARY extends ColumnType[BytesWritable, BytesWritable](10, 16) { buffer.put(v.getBytes(), 0, length) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { throw new UnsupportedOperationException() } @@ -367,7 +368,7 @@ object BINARY extends ColumnType[BytesWritable, BytesWritable](10, 16) { } } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: BytesWritable) { + override def extractInto(buffer: ByteBuffer, writable: BytesWritable) { val length = buffer.getInt() var b = _bytesFld.get(writable).asInstanceOf[Array[Byte]] if (b == null || b.length < length) { @@ -392,7 +393,7 @@ object GENERIC extends ColumnType[ByteStream.Output, ByteArrayRef](11, 16) { buffer.put(v.getData(), 0, length) } - override def extract(currentPos: Int, buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer) = { throw new UnsupportedOperationException() } @@ -400,7 +401,7 @@ object GENERIC extends ColumnType[ByteStream.Output, ByteArrayRef](11, 16) { o.asInstanceOf[ByteStream.Output] } - override def extractInto(currentPos: Int, buffer: ByteBuffer, writable: ByteArrayRef) { + override def extractInto(buffer: ByteBuffer, writable: ByteArrayRef) { val length = buffer.getInt() val a = new Array[Byte](length) buffer.get(a, 0, length) diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index 4870d85e..f3eda3d4 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -49,7 +49,7 @@ class DefaultDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extend override def hasNext = buffer.hasRemaining() override def next(): V = { - columnType.extractInto(buffer.position(), buffer, _current) + columnType.extractInto(buffer, _current) _current } } @@ -68,7 +68,7 @@ class RLDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends Ite override def next(): V = { if (_count == _run) { //next run - columnType.extractInto(buffer.position(), buffer, _current) + columnType.extractInto(buffer, _current) _run = buffer.getInt() _count = 1 } else { @@ -91,7 +91,7 @@ class DictDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends I var count = 0 while (count < size) { val writable = columnType.newWritable() - columnType.extractInto(buffer.position(), buffer, writable) + columnType.extractInto(buffer, writable) arr(count) = writable.asInstanceOf[V] count += 1 } diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index bd2f865f..9243158b 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -135,7 +135,7 @@ class RLE extends CompressionAlgorithm { writeOutRun() return } - val elem = t.extract(currentBuffer.position(), currentBuffer) + val elem = t.extract(currentBuffer) val newRun = if (currentRun == null) { (elem, 1) @@ -252,7 +252,7 @@ class DictionaryEncoding extends CompressionAlgorithm { // Write out the encoded values, each is represented by a short integer. while (b.hasRemaining()) { - val v = t.extract(b.position(), b) + val v = t.extract(b) compressedBuffer.putShort(_dictionary(v)) } diff --git a/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala b/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala index ec959bf7..d2110289 100644 --- a/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala +++ b/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala @@ -15,7 +15,7 @@ class ColumnTypeSuite extends FunSuite { a.foreach {i => buffer.putInt(i)} buffer.rewind() a.foreach {i => - val v = INT.extract(buffer.position(), buffer) + val v = INT.extract(buffer) assert(v == i) } buffer = ByteBuffer.allocate(32) @@ -32,7 +32,7 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() val writable = new IntWritable() a.foreach { i => - INT.extractInto(buffer.position(), buffer, writable) + INT.extractInto(buffer, writable) assert(writable.get == i) } @@ -46,7 +46,7 @@ class ColumnTypeSuite extends FunSuite { a.foreach {i => buffer.putShort(i)} buffer.rewind() a.foreach {i => - val v = SHORT.extract(buffer.position(), buffer) + val v = SHORT.extract(buffer) assert(v == i) } @@ -64,7 +64,7 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() val writable = new ShortWritable() a.foreach { i => - SHORT.extractInto(buffer.position(), buffer, writable) + SHORT.extractInto(buffer, writable) assert(writable.get == i) } } @@ -77,7 +77,7 @@ class ColumnTypeSuite extends FunSuite { a.foreach {i => buffer.putLong(i)} buffer.rewind() a.foreach {i => - val v = LONG.extract(buffer.position(), buffer) + val v = LONG.extract(buffer) assert(v == i) } @@ -95,7 +95,7 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() val writable = new LongWritable() a.foreach { i => - LONG.extractInto(buffer.position(), buffer, writable) + LONG.extractInto(buffer, writable) assert(writable.get == i) } } diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index 7d7d3ec7..74c04916 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -55,11 +55,11 @@ class CompressionAlgorithmSuite extends FunSuite { val compressedBuffer = rle.compress(b, STRING) assert(compressedBuffer.getInt() == STRING.typeID) assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals(new Text("abc"))) + assert(STRING.extract(compressedBuffer).equals(new Text("abc"))) assert(compressedBuffer.getInt() == 2) - assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals(new Text("efg"))) + assert(STRING.extract(compressedBuffer).equals(new Text("efg"))) assert(compressedBuffer.getInt() == 1) - assert(STRING.extract(compressedBuffer.position(), compressedBuffer).equals(new Text("abc"))) + assert(STRING.extract(compressedBuffer).equals(new Text("abc"))) assert(compressedBuffer.getInt() == 1) assert(!compressedBuffer.hasRemaining) } @@ -194,7 +194,7 @@ class CompressionAlgorithmSuite extends FunSuite { val dictionary = new HashMap[Short, T]() var count = 0 while (count < expectedDictSize) { - val v = u.extract(compressedBuffer.position(), compressedBuffer) + val v = u.extract(compressedBuffer) dictionary.put(dictionary.size.toShort, u.clone(v)) count += 1 } From 8b2063709fc334c4cc9bdbc81aaef57941853376 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 6 Sep 2013 20:54:53 +0800 Subject: [PATCH 015/331] Refactored column iterator interface to remove computeNext(). --- .../shark/memstore2/TablePartition.scala | 6 +- .../memstore2/TablePartitionIterator.scala | 1 - .../memstore2/column/ColumnBuilder.scala | 4 +- .../memstore2/column/ColumnIterator.scala | 27 ++-- .../column/CompressedColumnIterator.scala | 3 +- .../column/NullableColumnBuilder.scala | 36 +++-- .../column/NullableColumnIterator.scala | 18 ++- .../column/CompressionAlgorithmSuite.scala | 13 +- .../column/NullableColumnBuilderSuite.scala | 130 ++++++++++-------- .../column/NullableColumnIteratorSuite.scala | 6 +- .../scala/shark/util/BloomFilterSuite.scala | 8 +- 11 files changed, 127 insertions(+), 125 deletions(-) diff --git a/src/main/scala/shark/memstore2/TablePartition.scala b/src/main/scala/shark/memstore2/TablePartition.scala index 61235e85..ba8370a7 100644 --- a/src/main/scala/shark/memstore2/TablePartition.scala +++ b/src/main/scala/shark/memstore2/TablePartition.scala @@ -60,8 +60,6 @@ class TablePartition(private var _numRows: Long, private var _columns: Array[Byt buffer } - // TODO: Add column pruning to TablePartition for creating a TablePartitionIterator. - /** * Return an iterator for the partition. */ @@ -76,9 +74,9 @@ class TablePartition(private var _numRows: Long, private var _columns: Array[Byt def prunedIterator(columnsUsed: BitSet) = { val columnIterators: Array[ColumnIterator] = _columns.map { case buffer: ByteBuffer => - val iter = ColumnIterator.newIterator(buffer) - iter + ColumnIterator.newIterator(buffer) case _ => + // The buffer might be null if it is pruned in Tachyon. null } new TablePartitionIterator(_numRows, columnIterators, columnsUsed) diff --git a/src/main/scala/shark/memstore2/TablePartitionIterator.scala b/src/main/scala/shark/memstore2/TablePartitionIterator.scala index 71aabd7c..07312fc1 100644 --- a/src/main/scala/shark/memstore2/TablePartitionIterator.scala +++ b/src/main/scala/shark/memstore2/TablePartitionIterator.scala @@ -17,7 +17,6 @@ package shark.memstore2 -import java.nio.ByteBuffer import java.util.BitSet import shark.memstore2.column.ColumnIterator diff --git a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala index 2bb34404..cd00ae7d 100644 --- a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala +++ b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala @@ -65,8 +65,8 @@ trait ColumnBuilder[T] { protected def growIfNeeded(orig: ByteBuffer, size: Int): ByteBuffer = { val capacity = orig.capacity() if (orig.remaining() < size) { - //grow in steps of initial size - var additionalSize = capacity/8 + 1 + // grow in steps of initial size + val additionalSize = capacity / 8 + 1 var newSize = capacity + additionalSize if (additionalSize < size) { newSize = capacity + size diff --git a/src/main/scala/shark/memstore2/column/ColumnIterator.scala b/src/main/scala/shark/memstore2/column/ColumnIterator.scala index e0ca8520..57fae71d 100644 --- a/src/main/scala/shark/memstore2/column/ColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/ColumnIterator.scala @@ -22,31 +22,20 @@ import java.nio.{ByteBuffer, ByteOrder} trait ColumnIterator { - private var _initialized = false - + init() + def init() {} /** * Produces the next element of this iterator. */ - def next() { - if (!_initialized) { - init() - _initialized = true - } - computeNext() - } + def next() /** * Tests whether this iterator can provide another element. */ def hasNext: Boolean - /** - * Compute the next element so it is ready to be fetched using the current function. - */ - def computeNext() - /** * Return the current element. The operation should have no side-effect, i.e. it can be invoked * multiple times returning the same value. @@ -64,7 +53,7 @@ object Implicits { case DefaultCompressionType.typeID => DefaultCompressionType case RLECompressionType.typeID => RLECompressionType case DictionaryCompressionType.typeID => DictionaryCompressionType - case _ => throw new UnsupportedOperationException("Compression Type " + i) + case _ => throw new MemoryStoreException("Unknown compression type " + i) } implicit def intToColumnType(i: Int): ColumnType[_, _] = i match { @@ -80,6 +69,7 @@ object Implicits { case TIMESTAMP.typeID => TIMESTAMP case BINARY.typeID => BINARY case GENERIC.typeID => GENERIC + case _ => throw new MemoryStoreException("Unknown column type " + i) } } @@ -88,10 +78,14 @@ object ColumnIterator { import shark.memstore2.column.Implicits._ def newIterator(b: ByteBuffer): ColumnIterator = { + new NullableColumnIterator(b.duplicate().order(ByteOrder.nativeOrder())) + } + + def newNonNullIterator(b: ByteBuffer): ColumnIterator = { // The first 4 bytes in the buffer indicates the column type. val buffer = b.duplicate().order(ByteOrder.nativeOrder()) val columnType: ColumnType[_, _] = buffer.getInt() - val v = columnType match { + columnType match { case INT => new IntColumnIterator(buffer) case LONG => new LongColumnIterator(buffer) case FLOAT => new FloatColumnIterator(buffer) @@ -105,6 +99,5 @@ object ColumnIterator { case TIMESTAMP => new TimestampColumnIterator(buffer) case GENERIC => new GenericColumnIterator(buffer) } - new NullableColumnIterator(v, buffer) } } diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index f3eda3d4..e1a7cd3e 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -28,7 +28,8 @@ trait CompressedColumnIterator extends ColumnIterator { } } - override def computeNext() { + override def next() { + // TODO: can we remove the if branch? if (_decoder.hasNext) { _current = _decoder.next() } diff --git a/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala b/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala index 2b544f4e..68487569 100644 --- a/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala +++ b/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala @@ -7,19 +7,20 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector /** - * Builds a nullable column. The byte buffer of a nullable column contains - * the column type, followed by the null count and the index of nulls, followed - * finally by the non nulls. + * Builds a nullable column. The byte buffer of a nullable column contains: + * - 4 bytes for the null count (number of nulls) + * - positions for each null, in ascending order + * - the non-null data (column data type, compression type, data...) */ trait NullableColumnBuilder[T] extends ColumnBuilder[T] { private var _nulls: ByteBuffer = _ private var _pos: Int = _ - private var _nullCount:Int = _ + private var _nullCount: Int = _ override def initialize(initialSize: Int): ByteBuffer = { - _nulls = ByteBuffer.allocate(1024) + _nulls = ByteBuffer.allocate(1024) _nulls.order(ByteOrder.nativeOrder()) _pos = 0 _nullCount = 0 @@ -38,19 +39,16 @@ trait NullableColumnBuilder[T] extends ColumnBuilder[T] { } override def build(): ByteBuffer = { - val b = super.build() - if (_pos == 0) { - b - } else { - val v = _nulls.position() - _nulls.limit(v) - _nulls.rewind() - val newBuffer = ByteBuffer.allocate(b.limit + v + 4) - newBuffer.order(ByteOrder.nativeOrder()) - val colType= b.getInt() - newBuffer.putInt(colType).putInt(_nullCount).put(_nulls).put(b) - newBuffer.rewind() - newBuffer - } + val nonNulls = super.build() + val nullDataLen = _nulls.position() + _nulls.limit(nullDataLen) + _nulls.rewind() + + // 4 bytes for null count + null positions + non nulls + val newBuffer = ByteBuffer.allocate(4 + nullDataLen + nonNulls.limit) + newBuffer.order(ByteOrder.nativeOrder()) + newBuffer.putInt(_nullCount).put(_nulls).put(nonNulls) + newBuffer.rewind() + newBuffer } } \ No newline at end of file diff --git a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala index bd9ee77e..96ad0fda 100644 --- a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala @@ -9,7 +9,7 @@ import java.nio.ByteOrder * Reading of non nulls is delegated by setting the buffer position to the first * non null. */ -class NullableColumnIterator(delegate: ColumnIterator, buffer: ByteBuffer) extends ColumnIterator { +class NullableColumnIterator(buffer: ByteBuffer) extends ColumnIterator { private var _d: ByteBuffer = _ private var _nullCount: Int = _ private var _nulls = 0 @@ -18,17 +18,21 @@ class NullableColumnIterator(delegate: ColumnIterator, buffer: ByteBuffer) exten private var _currentNullIndex: Int = _ private var _pos = 0 + private var _delegate: ColumnIterator = _ + override def init() { _d = buffer.duplicate() _d.order(ByteOrder.nativeOrder()) _nullCount = _d.getInt() - buffer.position(buffer.position() + _nullCount * 4 + 4) _currentNullIndex = if (_nullCount > 0) _d.getInt() else Integer.MAX_VALUE _pos = 0 - delegate.init() + + // Move the buffer position to the non-null region. + buffer.position(buffer.position() + 4 + _nullCount * 4) + _delegate = ColumnIterator.newNonNullIterator(buffer) } - override def computeNext() { + override def next() { if (_pos == _currentNullIndex) { _nulls += 1 if (_nulls < _nullCount) { @@ -37,12 +41,12 @@ class NullableColumnIterator(delegate: ColumnIterator, buffer: ByteBuffer) exten _isNull = true } else { _isNull = false - delegate.computeNext() + _delegate.next() } _pos += 1 } - override def hasNext: Boolean = (_nulls < _nullCount) || delegate.hasNext + override def hasNext: Boolean = (_nulls < _nullCount) || _delegate.hasNext - def current: Object = if (_isNull) null else delegate.current + def current: Object = if (_isNull) null else _delegate.current } diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index 74c04916..7378e850 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -221,23 +221,24 @@ class CompressionAlgorithmSuite extends FunSuite { testList(longList, INT, Short.MaxValue - 1) } - test("RLE region") { + test("Uncompressed text") { val b = new StringColumnBuilder b.initialize(0) val oi = PrimitiveObjectInspectorFactory.javaStringObjectInspector - val lines = Array[String]("lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to", + val lines = Array[String]( + "lar deposits. blithely final packages cajole. regular waters are final requests.", "hs use ironic, even requests. s", "ges. thinly even pinto beans ca", "ly final courts cajole furiously final excuse", - "uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl" + "uickly special accounts cajole carefully blithely close requests. carefully final" ) lines.foreach { line => b.append(line, oi) } val newBuffer = b.build() - assert(newBuffer.getInt() == STRING.typeID) - assert(newBuffer.getInt() == RLECompressionType.typeID) - + assert(newBuffer.getInt() === 0) // null count + assert(newBuffer.getInt() === STRING.typeID) + assert(newBuffer.getInt() === DefaultCompressionType.typeID) } } diff --git a/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala b/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala index 86048d10..7b0a7baa 100644 --- a/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala +++ b/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala @@ -1,25 +1,22 @@ package shark.memstore2.column -import org.scalatest.FunSuite import org.apache.hadoop.io.Text import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.scalatest.FunSuite class NullableColumnBuilderSuite extends FunSuite { - test("Perf") { - val c = new StringColumnBuilder() - c.initialize(1024*1024*8) - val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector - Range(0, 1000000).foreach { i => - c.append(new Text("00000000000000000000000000000000" + i), oi) - } + test("Empty column") { + val c = new IntColumnBuilder() + c.initialize(4) val b = c.build() - val i = ColumnIterator.newIterator(b) - Range(0, 1000000).foreach { x => - i.next() - i.current - } + // # of nulls + assert(b.getInt() === 0) + // column type + assert(b.getInt() === INT.typeID) + assert(b.getInt() === DefaultCompressionType.typeID) + assert(!b.hasRemaining) } test("Buffer size auto growth") { @@ -35,8 +32,9 @@ class NullableColumnBuilderSuite extends FunSuite { c.append(null, oi) c.append(new Text("efg"), oi) val b = c.build() + b.position(4 + 4 * 4) val colType = b.getInt() - assert(colType == STRING.typeID) + assert(colType === STRING.typeID) } test("Null Strings") { @@ -48,22 +46,27 @@ class NullableColumnBuilderSuite extends FunSuite { c.append(new Text("b"), oi) c.append(null, oi) val b = c.build() - //expect first element is col type - assert(b.getInt() == STRING.typeID) - //next comes # of nulls - assert(b.getInt() == 2) - //typeID of first null is 1, that of second null is 3 - assert(b.getInt() == 1) - assert(b.getInt() == 3) + + // Number of nulls + assert(b.getInt() === 2) + + // First null position is 1, and then 3 + assert(b.getInt() === 1) + assert(b.getInt() === 3) + + // Column data type + assert(b.getInt() === STRING.typeID) - //next comes the compression type - assert(b.getInt() == -1) - assert(b.getInt() == 1) - assert(b.get() == 97) - assert(b.getInt() == 1) - assert(b.get() == 98) + // Compression type + assert(b.getInt() === DefaultCompressionType.typeID) + + // Data + assert(b.getInt() === 1) + assert(b.get() === 97) + assert(b.getInt() === 1) + assert(b.get() === 98) } - + test("Null Ints") { val c = new IntColumnBuilder() c.initialize(4) @@ -73,17 +76,34 @@ class NullableColumnBuilderSuite extends FunSuite { c.append(null, oi) c.append(56.asInstanceOf[Object], oi) val b = c.build() - //expect first element is col type - assert(b.getInt() == INT.typeID) - //next comes # of nulls - assert(b.getInt() == 2) - //typeID of first null is 1, that of second null is 3 - assert(b.getInt() == 1) - assert(b.getInt() == 2) - assert(b.getInt() == -1) - assert(b.getInt() == 123) + + // # of nulls and null positions + assert(b.getInt() === 2) + assert(b.getInt() === 1) + assert(b.getInt() === 2) + + // non nulls + assert(b.getInt() === INT.typeID) + assert(b.getInt() === DefaultCompressionType.typeID) + assert(b.getInt() === 123) + } + + test("Nullable Ints 2") { + val c = new IntColumnBuilder() + c.initialize(4) + val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector + Range(1, 1000).foreach { x => + c.append(x.asInstanceOf[Object], oi) + } + val b = c.build() + // null count + assert(b.getInt() === 0) + // column type + assert(b.getInt() === INT.typeID) + // compression type + assert(b.getInt() === DefaultCompressionType.typeID) } - + test("Null Longs") { val c = new LongColumnBuilder() c.initialize(4) @@ -93,26 +113,16 @@ class NullableColumnBuilderSuite extends FunSuite { c.append(null, oi) c.append(56L.asInstanceOf[Object], oi) val b = c.build() - //expect first element is col type - assert(b.getInt() == LONG.typeID) - //next comes # of nulls - assert(b.getInt() == 2) - //typeID of first null is 1, that of second null is 3 - assert(b.getInt() == 1) - assert(b.getInt() == 2) - assert(b.getInt() == -1) - assert(b.getLong() == 123L) - } - - test("Trigger RLE") { - val c = new IntColumnBuilder() - c.initialize(4) - val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector - Range(1, 1000).foreach { x => - c.append(x.asInstanceOf[Object], oi) - } - val b = c.build() - assert(b.getInt() == INT.typeID) - assert(b.getInt() == RLECompressionType.typeID) + + // # of nulls and null positions + assert(b.getInt() === 2) + assert(b.getInt() === 1) + assert(b.getInt() === 2) + + // non-nulls + assert(b.getInt() === LONG.typeID) + assert(b.getInt() === DefaultCompressionType.typeID) + assert(b.getLong() === 123L) } -} \ No newline at end of file + +} diff --git a/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala index f0eb9f95..da0622e9 100644 --- a/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala @@ -25,7 +25,7 @@ class NullableColumnIteratorSuite extends FunSuite { val b = c.build() val i = ColumnIterator.newIterator(b) Range(0, a.length).foreach { x => - if (x > 0) assert(true === i.hasNext) + if (x > 0) assert(i.hasNext) i.next() val v = i.current if (a(x) == null) { @@ -34,7 +34,7 @@ class NullableColumnIteratorSuite extends FunSuite { assert(v.toString == a(x).toString) } } - assert(false === i.hasNext) + assert(!i.hasNext) } test("Iterate Strings") { @@ -71,7 +71,7 @@ class NullableColumnIteratorSuite extends FunSuite { c.initialize(l.size) l.foreach { item => - c.append(item.asInstanceOf[AnyRef], oi) + c.append(item, oi) } val b = c.build() diff --git a/src/test/scala/shark/util/BloomFilterSuite.scala b/src/test/scala/shark/util/BloomFilterSuite.scala index 6126650f..31171d7e 100644 --- a/src/test/scala/shark/util/BloomFilterSuite.scala +++ b/src/test/scala/shark/util/BloomFilterSuite.scala @@ -5,13 +5,13 @@ import org.scalatest.FunSuite class BloomFilterSuite extends FunSuite{ test("Integer") { - val bf = new BloomFilter(0.03,1000000) - Range(0,1000000).foreach { + val bf = new BloomFilter(0.03, 1000000) + Range(0, 1000000).foreach { i => bf.add(i) } assert(bf.contains(333)) assert(bf.contains(678)) - assert(bf.contains(1200000) == false) + assert(!bf.contains(1200000)) } test("Integer FP") { @@ -26,12 +26,10 @@ class BloomFilterSuite extends FunSuite{ i => bf.contains(i*10) } val s = e.groupBy(x => x).map(x => (x._1, x._2.size)) - println(s) val t = s(true) val f = s(false) assert(f > 25 && f < 35) assert(t < 75 && t > 65) // expect false positive to be < 3 % and no false negatives - } } \ No newline at end of file From 64a43255a0bf8183168c64474fa78f4d78b3c203 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 7 Sep 2013 14:42:26 +0800 Subject: [PATCH 016/331] Improved size estimation for compression algorithms and added licensing header to all files. --- .../memstore2/column/ColumnBuilder.scala | 8 +-- .../memstore2/column/ColumnBuilders.scala | 47 +++++++++---- .../memstore2/column/ColumnIterators.scala | 23 ++++++- .../column/CompressedColumnIterator.scala | 21 +++++- .../column/CompressionAlgorithm.scala | 56 ++++++++++------ .../column/NullableColumnBuilder.scala | 19 +++++- .../column/NullableColumnIterator.scala | 19 +++++- .../memstore2/column/ColumnTypeSuite.scala | 61 +++++++++++------ .../CompressedColumnIteratorSuite.scala | 67 ++++++++++++++----- .../column/CompressionAlgorithmSuite.scala | 29 ++++++-- .../column/NullableColumnBuilderSuite.scala | 19 +++++- .../column/NullableColumnIteratorSuite.scala | 19 +++++- 12 files changed, 298 insertions(+), 90 deletions(-) diff --git a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala index cd00ae7d..4ca1f4e5 100644 --- a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala +++ b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala @@ -61,7 +61,7 @@ trait ColumnBuilder[T] { _buffer.order(ByteOrder.nativeOrder()) _buffer.putInt(t.typeID) } - + protected def growIfNeeded(orig: ByteBuffer, size: Int): ByteBuffer = { val capacity = orig.capacity() if (orig.remaining() < size) { @@ -82,7 +82,7 @@ trait ColumnBuilder[T] { } } -class DefaultColumnBuilder[T](val stats: ColumnStats[T], val t: ColumnType[T, _]) +class DefaultColumnBuilder[T](val stats: ColumnStats[T], val t: ColumnType[T, _]) extends CompressedColumnBuilder[T] with NullableColumnBuilder[T]{} @@ -105,7 +105,7 @@ trait CompressedColumnBuilder[T] extends ColumnBuilder[T] { override def build() = { val b = super.build() - + if (compressionSchemes.isEmpty) { new NoCompression().compress(b, t) } else { @@ -136,7 +136,7 @@ object ColumnBuilder { case PrimitiveCategory.BYTE => new ByteColumnBuilder case PrimitiveCategory.TIMESTAMP => new TimestampColumnBuilder case PrimitiveCategory.BINARY => new BinaryColumnBuilder - + // TODO: add decimal column. case _ => throw new MemoryStoreException( "Invalid primitive object inspector category" + columnOi.getCategory) diff --git a/src/main/scala/shark/memstore2/column/ColumnBuilders.scala b/src/main/scala/shark/memstore2/column/ColumnBuilders.scala index 593f8685..6cee1359 100644 --- a/src/main/scala/shark/memstore2/column/ColumnBuilders.scala +++ b/src/main/scala/shark/memstore2/column/ColumnBuilders.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.ByteBuffer @@ -12,18 +29,6 @@ import shark.execution.serialization.KryoSerializer import shark.memstore2.column.ColumnStats._ -class GenericColumnBuilder(oi: ObjectInspector) - extends DefaultColumnBuilder[ByteStream.Output](new NoOpStats(), GENERIC) { - - override def initialize(initialSize: Int):ByteBuffer = { - val buffer = super.initialize(initialSize) - val objectInspectorSerialized = KryoSerializer.serialize(oi) - buffer.putInt(objectInspectorSerialized.size) - buffer.put(objectInspectorSerialized) - buffer - } -} - class BooleanColumnBuilder extends DefaultColumnBuilder[Boolean](new BooleanColumnStats(), BOOLEAN) class IntColumnBuilder extends DefaultColumnBuilder[Int](new IntColumnStats(), INT) @@ -45,4 +50,20 @@ class TimestampColumnBuilder class BinaryColumnBuilder extends DefaultColumnBuilder[BytesWritable](new NoOpStats(), BINARY) -class VoidColumnBuilder extends DefaultColumnBuilder[Void](new NoOpStats(), VOID) \ No newline at end of file +class VoidColumnBuilder extends DefaultColumnBuilder[Void](new NoOpStats(), VOID) + +/** + * Generic columns that we can serialize, including maps, structs, and other complex types. + */ +class GenericColumnBuilder(oi: ObjectInspector) + extends DefaultColumnBuilder[ByteStream.Output](new NoOpStats(), GENERIC) { + + // Complex data types cannot be null. Override the initialize in NullableColumnBuilder. + override def initialize(initialSize: Int): ByteBuffer = { + val buffer = super.initialize(initialSize) + val objectInspectorSerialized = KryoSerializer.serialize(oi) + buffer.putInt(objectInspectorSerialized.size) + buffer.put(objectInspectorSerialized) + buffer + } +} diff --git a/src/main/scala/shark/memstore2/column/ColumnIterators.scala b/src/main/scala/shark/memstore2/column/ColumnIterators.scala index be9902b5..3060b5b1 100644 --- a/src/main/scala/shark/memstore2/column/ColumnIterators.scala +++ b/src/main/scala/shark/memstore2/column/ColumnIterators.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.ByteBuffer @@ -32,9 +49,9 @@ class BinaryColumnIterator(buffer: ByteBuffer) extends DefaultColumnIterator(buf class StringColumnIterator(buffer: ByteBuffer) extends DefaultColumnIterator(buffer, STRING) class GenericColumnIterator(buffer: ByteBuffer) extends DefaultColumnIterator(buffer, GENERIC) { - + private var _obj: LazyObject[_] = _ - + override def init() { super.init() val oiSize = buffer.getInt() @@ -43,7 +60,7 @@ class GenericColumnIterator(buffer: ByteBuffer) extends DefaultColumnIterator(bu val oi = KryoSerializer.deserialize[ObjectInspector](oiSerialized) _obj = LazyFactory.createLazyObject(oi) } - + override def current = { val v = super.current.asInstanceOf[ByteArrayRef] _obj.init(v, 0, v.getData().length) diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index e1a7cd3e..5acd75a6 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.ByteBuffer @@ -59,13 +76,13 @@ class DefaultDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extend * Run Length Decoder, decodes data compressed in RLE format of [element, length] */ class RLDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends Iterator[V] { - + private var _run: Int = _ private var _count: Int = 0 private val _current: V = columnType.newWritable() override def hasNext = buffer.hasRemaining() - + override def next(): V = { if (_count == _run) { //next run diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index 9243158b..efab4bc5 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -24,8 +24,19 @@ trait CompressionAlgorithm { /** * Return compression ratio between 0 and 1, smaller score imply higher compressibility. + * This is used to pick the compression algorithm to apply at runtime. */ - def compressionRatio: Double + def compressionRatio: Double = compressedSize.toDouble / uncompressedSize.toDouble + + /** + * The uncompressed size of the input data. + */ + def uncompressedSize: Int + + /** + * Estimation of the data size once compressed. + */ + def compressedSize: Int /** * Compress the given buffer and return the compressed data as a new buffer. @@ -42,8 +53,11 @@ object RLECompressionType extends CompressionType(0) object DictionaryCompressionType extends CompressionType(1) - +/** + * An no-op compression. + */ class NoCompression extends CompressionAlgorithm { + override def compressionType = DefaultCompressionType override def supportsType(t: ColumnType[_,_]) = true @@ -52,6 +66,10 @@ class NoCompression extends CompressionAlgorithm { override def compressionRatio: Double = 1.0 + override def uncompressedSize: Int = 0 + + override def compressedSize: Int = 0 + override def compress[T](b: ByteBuffer, t: ColumnType[T, _]) = { val len = b.limit() val newBuffer = ByteBuffer.allocate(len + 4) @@ -92,13 +110,14 @@ class RLE extends CompressionAlgorithm { // This is the very first run. _prev = t.clone(v) _run = 1 + _compressedSize += s + 4 } else { if (_prev.equals(v)) { // Add one to the current run's length. _run += 1 } else { // Start a new run. Update the current run length. - _compressedSize += (t.actualSize(_prev.asInstanceOf[T]) + 4) + _compressedSize += s + 4 _prev = t.clone(v) _run = 1 } @@ -106,17 +125,15 @@ class RLE extends CompressionAlgorithm { _uncompressedSize += s } + override def uncompressedSize: Int = _uncompressedSize + // Note that we don't actually track the size of the last run into account to simplify the // logic a little bit. - override def compressionRatio = _compressedSize / (_uncompressedSize + 0.0) + override def compressedSize: Int = _compressedSize override def compress[T](b: ByteBuffer, t: ColumnType[T,_]): ByteBuffer = { - // Add the size of the last run to the _size - if (_prev != null) { - _compressedSize += t.actualSize(_prev.asInstanceOf[T]) + 4 - } - - val compressedBuffer = ByteBuffer.allocate(_compressedSize + 4 + 4) + // Leave 4 extra bytes for column type and another 4 for compression type. + val compressedBuffer = ByteBuffer.allocate(4 + 4 + _compressedSize) compressedBuffer.order(ByteOrder.nativeOrder()) compressedBuffer.putInt(b.getInt()) compressedBuffer.putInt(compressionType.typeID) @@ -216,20 +233,18 @@ class DictionaryEncoding extends CompressionAlgorithm { } } + override def uncompressedSize: Int = _uncompressedSize + /** - * Return the compression ratio if encoded with dictionary encoding. If the dictionary + * Return the compressed data size if encoded with dictionary encoding. If the dictionary * cardinality (i.e. the number of distinct elements) is bigger than 32K, we return an - * arbitrary number greater than 1.0. + * a really large number. */ - override def compressionRatio: Double = compressedSize / (_uncompressedSize + 0.0) - - private def compressedSize: Int = { + override def compressedSize: Int = { // Total compressed size = // size of the dictionary + - // the number of elements * dictionary encoded size (short) + - // an integer for compression type - // an integer for column type - if (_overflow) Int.MaxValue else _dictionarySize + _count * indexSize + 4 + 4 + // the number of elements * dictionary encoded size (short) + if (_overflow) Int.MaxValue else _dictionarySize + _count * indexSize } override def compress[T](b: ByteBuffer, t: ColumnType[T, _]): ByteBuffer = { @@ -239,7 +254,8 @@ class DictionaryEncoding extends CompressionAlgorithm { } // Create a new buffer and store the compression type and column type. - val compressedBuffer = ByteBuffer.allocate(compressedSize) + // Leave 4 extra bytes for column type and another 4 for compression type. + val compressedBuffer = ByteBuffer.allocate(4 + 4 + compressedSize) compressedBuffer.order(ByteOrder.nativeOrder()) compressedBuffer.putInt(b.getInt()) compressedBuffer.putInt(compressionType.typeID) diff --git a/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala b/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala index 68487569..53472c7b 100644 --- a/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala +++ b/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.ByteBuffer @@ -15,7 +32,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector trait NullableColumnBuilder[T] extends ColumnBuilder[T] { private var _nulls: ByteBuffer = _ - + private var _pos: Int = _ private var _nullCount: Int = _ diff --git a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala index 96ad0fda..49e0eb20 100644 --- a/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/NullableColumnIterator.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.ByteBuffer @@ -45,7 +62,7 @@ class NullableColumnIterator(buffer: ByteBuffer) extends ColumnIterator { } _pos += 1 } - + override def hasNext: Boolean = (_nulls < _nullCount) || _delegate.hasNext def current: Object = if (_isNull) null else _delegate.current diff --git a/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala b/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala index d2110289..1ea2f7a6 100644 --- a/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala +++ b/src/test/scala/shark/memstore2/column/ColumnTypeSuite.scala @@ -1,11 +1,30 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column -import org.scalatest.FunSuite import java.nio.ByteBuffer + import org.apache.hadoop.io.IntWritable import org.apache.hadoop.io.LongWritable import org.apache.hadoop.hive.serde2.io._ +import org.scalatest.FunSuite + class ColumnTypeSuite extends FunSuite { test("Int") { @@ -14,30 +33,30 @@ class ColumnTypeSuite extends FunSuite { var a: Seq[Int] = Array[Int](35, 67, 899, 4569001) a.foreach {i => buffer.putInt(i)} buffer.rewind() - a.foreach {i => + a.foreach {i => val v = INT.extract(buffer) assert(v == i) } buffer = ByteBuffer.allocate(32) a = Range(0, 4) - a.foreach { i => - INT.append(i, buffer) + a.foreach { i => + INT.append(i, buffer) } buffer.rewind() a.foreach { i => assert(buffer.getInt() == i)} - + buffer = ByteBuffer.allocate(32) a =Range(0,4) a.foreach { i => buffer.putInt(i)} buffer.rewind() val writable = new IntWritable() - a.foreach { i => + a.foreach { i => INT.extractInto(buffer, writable) assert(writable.get == i) } - + } - + test("Short") { assert(SHORT.defaultSize == 2) assert(SHORT.actualSize(8) == 2) @@ -45,30 +64,30 @@ class ColumnTypeSuite extends FunSuite { var a = Array[Short](35, 67, 87, 45) a.foreach {i => buffer.putShort(i)} buffer.rewind() - a.foreach {i => + a.foreach {i => val v = SHORT.extract(buffer) assert(v == i) } - + buffer = ByteBuffer.allocate(32) a = Array[Short](0,1,2,3) - a.foreach { i => - SHORT.append(i, buffer) + a.foreach { i => + SHORT.append(i, buffer) } buffer.rewind() a.foreach { i => assert(buffer.getShort() == i)} - + buffer = ByteBuffer.allocate(32) a =Array[Short](0,1,2,3) a.foreach { i => buffer.putShort(i)} buffer.rewind() val writable = new ShortWritable() - a.foreach { i => + a.foreach { i => SHORT.extractInto(buffer, writable) assert(writable.get == i) } } - + test("Long") { assert(LONG.defaultSize == 8) assert(LONG.actualSize(45L) == 8) @@ -76,25 +95,25 @@ class ColumnTypeSuite extends FunSuite { var a = Array[Long](35L, 67L, 8799000880L, 45000999090L) a.foreach {i => buffer.putLong(i)} buffer.rewind() - a.foreach {i => + a.foreach {i => val v = LONG.extract(buffer) assert(v == i) } - + buffer = ByteBuffer.allocate(32) a = Array[Long](0,1,2,3) - a.foreach { i => - LONG.append(i, buffer) + a.foreach { i => + LONG.append(i, buffer) } buffer.rewind() a.foreach { i => assert(buffer.getLong() == i)} - + buffer = ByteBuffer.allocate(32) a =Array[Long](0,1,2,3) a.foreach { i => buffer.putLong(i)} buffer.rewind() val writable = new LongWritable() - a.foreach { i => + a.foreach { i => LONG.extractInto(buffer, writable) assert(writable.get == i) } diff --git a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala index 316e7cc8..3096ce28 100644 --- a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.ByteBuffer @@ -25,7 +42,7 @@ class CompressedColumnIteratorSuite extends FunSuite { l: Seq[T], t: ColumnType[T, _], algo: CompressionAlgorithm, - compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b, + expectedCompressedSize: Int, shouldNotCompress: Boolean = false) { val b = ByteBuffer.allocate(1024 + (3 * 40 * l.size)) @@ -38,6 +55,11 @@ class CompressedColumnIteratorSuite extends FunSuite { b.limit(b.position()) b.rewind() + info("compressed size: %d, uncompressed size: %d, compression ratio %f".format( + algo.compressedSize, algo.uncompressedSize, algo.compressionRatio)) + + assert(algo.compressedSize === expectedCompressedSize) + if (shouldNotCompress) { assert(algo.compressionRatio >= 1.0) } else { @@ -56,7 +78,7 @@ class CompressedColumnIteratorSuite extends FunSuite { l.foreach { x => iter.next() - assert(compareFunc(t.get(iter.current, oi), x)) + assert(t.get(iter.current, oi) === x) } // Make sure we reach the end of the iterator. @@ -65,60 +87,71 @@ class CompressedColumnIteratorSuite extends FunSuite { } test("RLE Boolean") { - testList(Seq[Boolean](true, true, false, true), BOOLEAN, new RLE()) + // 3 runs: (1+4)*3 + val bools = Seq(true, true, false, true, true, true, true, true, true, true, true, true) + testList(bools, BOOLEAN, new RLE, 15) } test("RLE Byte") { - testList(Seq[Byte](10, 10, 20, 10), BYTE, new RLE()) + // 3 runs: (1+4)*3 + testList(Seq[Byte](10, 10, 10, 10, 10, 10, 10, 10, 10, 20, 10), BYTE, new RLE, 15) } test("RLE Short") { - testList(Seq[Short](10, 10, 10, 20000, 20000, 20000, 500, 500, 500, 500), SHORT, new RLE()) + // 3 runs: (2+4)*3 + testList(Seq[Short](10, 10, 10, 20000, 20000, 20000, 500, 500, 500, 500), SHORT, new RLE, 18) } test("RLE Int") { - testList(Seq[Int](1000000, 1000000, 1000000, 1000000, 900000, 99), INT, new RLE()) + // 3 runs: (4+4)*3 + testList(Seq[Int](1000000, 1000000, 1000000, 1000000, 900000, 99), INT, new RLE, 24) } test("RLE Long") { + // 2 runs: (8+4)*3 val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) - testList(longs, LONG, new RLE()) + testList(longs, LONG, new RLE, 24) } test("RLE String") { + // 3 runs: (4+4+4) + (4+1+4) + (4+1+4) = 30 val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "!", "!").map(s => new Text(s)) - testList(strs, STRING, new RLE(), (a: Text, b: Text) => a.equals(b)) + testList(strs, STRING, new RLE, 30) } test("Dictionary Encoded Int") { - testList(Seq[Int](1000000, 1000000, 1000000, 1000000, 900000, 99), INT, new DictionaryEncoding) + // dict len + 3 distinct values + 7 values = 4 + 3*4 + 7*2 = 30 + val ints = Seq[Int](1000000, 1000000, 99, 1000000, 1000000, 900000, 99) + testList(ints, INT, new DictionaryEncoding, 30) } test("Dictionary Encoded Long") { + // dict len + 2 distinct values + 7 values = 4 + 2*8 + 7*2 = 34 val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) - testList(longs, LONG, new DictionaryEncoding) + testList(longs, LONG, new DictionaryEncoding, 34) } test("Dictionary Encoded String") { - val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "!", "!").map(s => new Text(s)) - testList(strs, STRING, new DictionaryEncoding, (a: Text, b: Text) => a.equals(b), - shouldNotCompress = false) + // dict len + 3 distinct values + 8 values = 4 + (4+4) + (4+1) + (4+1) + 8*2 = + val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "e", "!", "!").map(s => new Text(s)) + testList(strs, STRING, new DictionaryEncoding, 38, shouldNotCompress = false) } test("Dictionary Encoding at limit of unique values") { val ints = Range(0, Short.MaxValue - 1).flatMap(i => Iterator(i, i, i)) - testList(ints, INT, new DictionaryEncoding) + val expectedLen = 4 + (Short.MaxValue - 1) * 4 + 2 * (Short.MaxValue - 1) * 3 + testList(ints, INT, new DictionaryEncoding, expectedLen) } test("Dictionary Encoding - should not compress") { val ints = Range(0, Short.MaxValue.toInt) - testList(ints, INT, new DictionaryEncoding, (a: Int, b: Int) => a == b, - shouldNotCompress = true) + testList(ints, INT, new DictionaryEncoding, Int.MaxValue, shouldNotCompress = true) } test("RLE - should not compress") { val ints = Range(0, Short.MaxValue.toInt + 1) - testList(ints, INT, new RLE, (a: Int, b: Int) => a == b, shouldNotCompress = true) + val expectedLen = (Short.MaxValue.toInt + 1) * (4 + 4) + testList(ints, INT, new RLE, expectedLen, shouldNotCompress = true) } } diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index 7378e850..3059b40c 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.{ByteBuffer, ByteOrder} @@ -84,7 +101,7 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 1) assert(!compressedBuffer.hasRemaining) } - + test("RLE int single run") { val b = ByteBuffer.allocate(4008) b.order(ByteOrder.nativeOrder()) @@ -103,7 +120,7 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 1000) assert(!compressedBuffer.hasRemaining) } - + test("RLE long single run") { val b = ByteBuffer.allocate(8008) b.order(ByteOrder.nativeOrder()) @@ -122,7 +139,7 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 1000) assert(!compressedBuffer.hasRemaining) } - + test("RLE int 3 runs") { val b = ByteBuffer.allocate(4008) b.order(ByteOrder.nativeOrder()) @@ -168,7 +185,7 @@ class CompressionAlgorithmSuite extends FunSuite { assert(compressedBuffer.getInt() == 1000000) assert(!compressedBuffer.hasRemaining) } - + test("Dictionary Encoding") { def testList[T]( @@ -200,7 +217,7 @@ class CompressionAlgorithmSuite extends FunSuite { } assert(dictionary.get(0).get.equals(l(0))) assert(dictionary.get(1).get.equals(l(2))) - l.foreach { x => + l.foreach { x => val y = dictionary.get(compressedBuffer.getShort()).get assert(compareFunc(y, x)) } @@ -220,7 +237,7 @@ class CompressionAlgorithmSuite extends FunSuite { assert(longList.size === (8 + 2*(Short.MaxValue-1))) testList(longList, INT, Short.MaxValue - 1) } - + test("Uncompressed text") { val b = new StringColumnBuilder b.initialize(0) diff --git a/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala b/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala index 7b0a7baa..92e58760 100644 --- a/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala +++ b/src/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import org.apache.hadoop.io.Text @@ -56,7 +73,7 @@ class NullableColumnBuilderSuite extends FunSuite { // Column data type assert(b.getInt() === STRING.typeID) - + // Compression type assert(b.getInt() === DefaultCompressionType.typeID) diff --git a/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala index da0622e9..614fc625 100644 --- a/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory @@ -63,7 +80,7 @@ class NullableColumnIteratorSuite extends FunSuite { assert(i.current == null) assert(false === i.hasNext) } - + test("Iterate Ints") { def testList(l: Seq[AnyRef]) { val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector From 34797816366da8f718c99c11a5f0531933bb0e29 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 6 Sep 2013 14:03:49 -0700 Subject: [PATCH 017/331] Add a MemoryTableDesc wrapper around a table RDD, or partition RDDs if the table is Hive-partitioned. --- .../shark/memstore2/MemoryTableDesc.scala | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) create mode 100644 src/main/scala/shark/memstore2/MemoryTableDesc.scala diff --git a/src/main/scala/shark/memstore2/MemoryTableDesc.scala b/src/main/scala/shark/memstore2/MemoryTableDesc.scala new file mode 100644 index 00000000..92dff91f --- /dev/null +++ b/src/main/scala/shark/memstore2/MemoryTableDesc.scala @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.util.{HashMap => JavaHashMap} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.Map + +import org.apache.spark.rdd.RDD + + +/** + * A container for table metadata specific to Shark and Spark. Currently, this is a lightweight + * wrapper around either an RDD or multiple RDDs if the Shark table is Hive-partitioned. + * Note that a Hive-partition of a table is different from an RDD partition. Each Hive-partition + * is stored as a subdirectory of the table subdirectory in the warehouse directory + * (e.g. /user/hive/warehouse). So, every Hive-Partition is loaded into Shark as an RDD, and is + * cached as one if the user-specifies it. + */ +private[shark] class MemoryTableDesc(tableName: String, isHivePartitioned: Boolean) { + + /** Should be used if the table is not Hive-partitioned. */ + private var _tableRDD: RDD[_] = _ + + /** + * Should be used if a cached table is Hive-partitioned. + */ + private val _hivePartitionRDDs: Map[String, RDD[_]] = + if (isHivePartitioned) { new JavaHashMap[String, RDD[_]]() } else { null } + + def tableRDD: RDD[_] = { + assert ( + !isHivePartitioned, + "Table " + tableName + " is Hive-partitioned. Use MemoryTableDesc::hivePartitionRDDs() " + + "to get RDDs corresponding to partition columns" + ) + return _tableRDD + } + + def tableRDD_= (value: RDD[_]) { + assert( + !isHivePartitioned, + "Table " + tableName + " is Hive-partitioned. Pass in a map of pairs " + + "the 'hivePartitionRDDs =' setter." + ) + _tableRDD = value + } + + def hivePartitionRDDs: Map[String, RDD[_]] = { + assert(isHivePartitioned, + "Table " + tableName + " is not Hive-partitioned. Use tableRDD() to get its RDD.") + _hivePartitionRDDs + } + + def addHivePartitionRDD(partitionKey: String, rdd: RDD[_]) { + assert(isHivePartitioned, + "Table " + tableName + " is not Hive-partitioned. Use the 'tableRDD =' setter.") + _hivePartitionRDDs(partitionKey) = rdd + } +} \ No newline at end of file From e793743fa20333b630814b0929c839b1a854ec49 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 14:30:15 -0700 Subject: [PATCH 018/331] Comment in SharkSemanticAnalyzer about handling CREATE TABLE and CTAS. --- .../shark/parse/SharkSemanticAnalyzer.scala | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 2dfbc695..65850d40 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -78,25 +78,50 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with //TODO: can probably reuse Hive code for this // analyze create table command var cacheMode = CacheType.NONE - var isCTAS = false var shouldReset = false + // These are set when parsing the command statement AST for a CREATE TABLE. + var isCTAS = false + var isPartitioned = false + if (ast.getToken().getType() == HiveParser.TOK_CREATETABLE) { super.analyzeInternal(ast) for (ch <- ast.getChildren) { ch.asInstanceOf[ASTNode].getToken.getType match { case HiveParser.TOK_QUERY => { isCTAS = true + // Set the child ASTNode, which will be passed as an argument to + // SemanticAnalzyer#doPhase1(). child = ch.asInstanceOf[ASTNode] } + case HiveParser.TOK_TABLEPARTCOLS => { + // If the table that will be created is Hive-partitioned and should be cached, then + // metadata will be initialized in the initializeCachedTableMetadata() call below. + isPartitionedTable = true + // Get the partitioning columns. In Hive, CREATE TABLE ... [PARTITIONED BY] ... + // is handled by a DDLTask (created by the Hive SemanticAnalyzer's genMapRedTasks and + // not to be confused with the Hive DDLSemanticAnalyzer, which handles ALTER/DROP table + // (among other things). Since creating tables in Shark doesn't involve too much + // overhead (e.g. we don't support indexing), just update the Shark + // MemoryMetaDataManager in this method (during the semantic analysis phase). + // TODO(harvey): a Shark-specific DDLTask might be needed once indexing is supported. + partitionColumns = BaseSemanticAnalyzer.getColumns((ASTNode) child.getChild(0), false); + } case _ => Unit } } - // If the table descriptor can be null if the CTAS has an - // "if not exists" condition. + // The table descriptor can be NULL if the command is a ... + // 1) syntactically valid CREATE TABLE statement. The table specified may or may not already + // exist. If the table already exists, then an exception is thrown by the DDLTask that's + // executed after semantic analysis. + // 2) valid CTAS statement with an IF NOT EXISTS condition, and the specified table already + // exists. If the table to-be-created already exists, and the CTAS statement does not + // have an IF NOT EXISTS condition, then an exception will be thrown by the parent + // SemanticAnalzyer's analyzeInternal() call above. val createTableDesc = getParseContext.getQB.getTableDesc + if (!isCTAS || createTableDesc == null) { return } else { @@ -108,7 +133,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // There are two cases that will enable caching: // 1) Table name includes "_cached" or "_tachyon". // 2) The "shark.cache" table property is "true", or the string representation of a supported - // cache mode (heap, Tachyon). + // cache mode (heap, Tachyon). cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) // Continue planning based on the 'cacheMode' read. if (cacheMode == CacheType.HEAP || From 43cfb213741da7a6c4b003eee4c475df03076526 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 15:43:28 -0700 Subject: [PATCH 019/331] Move CREATE TABLE/CTAS analysis to a new method. This includes adding a new QueryBlock that's used to store detected cache modes. Also, renamed MemoryTableDesc to MemoryTable. --- src/main/scala/shark/SharkEnv.scala | 2 +- .../memstore2/MemoryMetadataManager.scala | 66 +++++-- ...emoryTableDesc.scala => MemoryTable.scala} | 16 +- src/main/scala/shark/parse/QueryBlock.scala | 36 ++++ .../shark/parse/SharkSemanticAnalyzer.scala | 176 +++++++++++------- 5 files changed, 194 insertions(+), 102 deletions(-) rename src/main/scala/shark/memstore2/{MemoryTableDesc.scala => MemoryTable.scala} (81%) create mode 100644 src/main/scala/shark/parse/QueryBlock.scala diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 55d646e0..f082fee4 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -114,7 +114,7 @@ object SharkEnv extends LogHelper { val addedFiles = HashSet[String]() val addedJars = HashSet[String]() - def unpersist(key: String): Option[RDD[_]] = { + def unpersist(key: String) { if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(key)) { if (SharkEnv.tachyonUtil.dropTable(key)) { logInfo("Table " + key + " was deleted from Tachyon."); diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index c180dd40..75d190e4 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -30,19 +30,39 @@ import shark.SharkConfVars class MemoryMetadataManager { - private val _keyToRdd: ConcurrentMap[String, RDD[_]] = - new ConcurrentHashMap[String, RDD[_]]() + private val _keyToMemoryTable: ConcurrentMap[String, MemoryTable] = + new ConcurrentHashMap[String, MemoryTable]() + // TODO(harvey): Support stats for Hive-partitioned tables. private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - def contains(key: String) = _keyToRdd.contains(key.toLowerCase) + def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) def put(key: String, rdd: RDD[_]) { - _keyToRdd(key.toLowerCase) = rdd + if (!contains(key)) { + _keyToMemoryTable(key.toLowerCase) = new MemoryTable(key, false /* isHivePartitioned */) + } + _keyToMemoryTable(key.toLowerCase).tableRDD = rdd + } + + def putHivePartition(key: String, partitionColumn: String, rdd: RDD[_]) { + if (!contains(key)) { + _keyToMemoryTable(key.toLowerCase) = new MemoryTable(key, true /* isHivePartitioned */) + } + _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumn) = rdd } - def get(key: String): Option[RDD[_]] = _keyToRdd.get(key.toLowerCase) + def get(key: String): Option[RDD[_]] = { + _keyToMemoryTable.get(key.toLowerCase) match { + case Some(memoryTable) => return Some(memoryTable.tableRDD) + case None => return None + } + } + + def getHivePartition(key: String, partitionColumn: String): RDD[_] = { + return _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumn) + } def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { _keyToStats.put(key.toLowerCase, stats) @@ -56,19 +76,18 @@ class MemoryMetadataManager { * Find all keys that are strings. Used to drop tables after exiting. */ def getAllKeyStrings(): Seq[String] = { - _keyToRdd.keys.collect { case k: String => k } toSeq + _keyToMemoryTable.keys.collect { case k: String => k } toSeq } /** - * Used to drop an RDD from the Spark in-memory cache and/or disk. All metadata - * (e.g. entry in '_keyToStats') about the RDD that's tracked by Shark is deleted as well. + * Used to drop a table from the Spark in-memory cache and/or disk. All metadata + * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) tracked by Shark is deleted + * as well. * - * @param key Used to fetch the an RDD value from '_keyToRDD'. - * @return Option::isEmpty() is true if there is no RDD value corresponding to 'key' in - * '_keyToRDD'. Otherwise, returns a reference to the RDD that was unpersist()'ed. + * @param key Name of the table to drop. */ - def unpersist(key: String): Option[RDD[_]] = { - def unpersistRDD(rdd: RDD[_]): Unit = { + def unpersist(key: String) { + def unpersistRDD(rdd: RDD[_]) { rdd match { case u: UnionRDD[_] => { // Recursively unpersist() all RDDs that compose the UnionRDD. @@ -80,16 +99,23 @@ class MemoryMetadataManager { case r => r.unpersist() } } - // Remove RDD's entry from Shark metadata. This also fetches a reference to the RDD object - // corresponding to the argument for 'key'. - val rddValue = _keyToRdd.remove(key.toLowerCase()) + def unpersistMemoryTable(memoryTable: MemoryTable) { + if (memoryTable.isHivePartitioned) { + // unpersist() all RDDs for all Hive-partitions. + memoryTable.hivePartitionRDDs.mapValues(unpersistRDD(_)) + } else { + unpersistRDD(memoryTable.tableRDD) + } + } + // Remove MemoryTable's entry from Shark metadata. + val memoryTableValue = _keyToMemoryTable.remove(key.toLowerCase()) _keyToStats.remove(key) - // Unpersist the RDD using the nested helper fn above. - rddValue match { - case Some(rdd) => unpersistRDD(rdd) + + // Unpersist the MemoryTable using the nested helper functions above. + memoryTableValue match { + case Some(memoryTable) => unpersistMemoryTable(memoryTable) case None => Unit } - rddValue } } diff --git a/src/main/scala/shark/memstore2/MemoryTableDesc.scala b/src/main/scala/shark/memstore2/MemoryTable.scala similarity index 81% rename from src/main/scala/shark/memstore2/MemoryTableDesc.scala rename to src/main/scala/shark/memstore2/MemoryTable.scala index 92dff91f..b2e0206d 100644 --- a/src/main/scala/shark/memstore2/MemoryTableDesc.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -32,14 +32,18 @@ import org.apache.spark.rdd.RDD * is stored as a subdirectory of the table subdirectory in the warehouse directory * (e.g. /user/hive/warehouse). So, every Hive-Partition is loaded into Shark as an RDD, and is * cached as one if the user-specifies it. + * + * TODO(harvey): It could be useful to make MemoryTable a parent class, and have other table types, + * such as HivePartitionedTable or TachyonTable, subclass it. For now, there isn't + * too much metadata to track, so it should be okay to have a single MemoryTable. */ -private[shark] class MemoryTableDesc(tableName: String, isHivePartitioned: Boolean) { +private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: Boolean) { - /** Should be used if the table is not Hive-partitioned. */ + /** Should only be used if the table is not Hive-partitioned. */ private var _tableRDD: RDD[_] = _ /** - * Should be used if a cached table is Hive-partitioned. + * Should only be used if a cached table is Hive-partitioned. */ private val _hivePartitionRDDs: Map[String, RDD[_]] = if (isHivePartitioned) { new JavaHashMap[String, RDD[_]]() } else { null } @@ -67,10 +71,4 @@ private[shark] class MemoryTableDesc(tableName: String, isHivePartitioned: Boole "Table " + tableName + " is not Hive-partitioned. Use tableRDD() to get its RDD.") _hivePartitionRDDs } - - def addHivePartitionRDD(partitionKey: String, rdd: RDD[_]) { - assert(isHivePartitioned, - "Table " + tableName + " is not Hive-partitioned. Use the 'tableRDD =' setter.") - _hivePartitionRDDs(partitionKey) = rdd - } } \ No newline at end of file diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala new file mode 100644 index 00000000..09d4fb9a --- /dev/null +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.parse + +import org.apache.hadoop.hive.ql.parse.{QB => HiveQueryBlock} + +import shark.memstore2.CacheType +import shark.memstore2.CacheType._ + +class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) + extends HiveQueryBlock(outerID, alias, isSubQuery) { + + // The CacheType for the table of a CREATE TABLE/CTAS, + private var cacheMode = CacheType.NONE + + def setCacheMode(cacheModeToSet: CacheType) { + cacheMode = cacheModeToSet + } + + def getCacheMode(): CacheType = cacheMode +} diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 65850d40..eacb9690 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -66,97 +66,41 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with override def analyzeInternal(ast: ASTNode): Unit = { reset() - val qb = new QB(null, null, false) + val qb = new QueryBlock(null, null, false) val pctx = getParseContext() pctx.setQB(qb) pctx.setParseTree(ast) init(pctx) + // The ASTNode that will be analyzed by SemanticAnalzyer#doPhase1(). var child: ASTNode = ast logInfo("Starting Shark Semantic Analysis") //TODO: can probably reuse Hive code for this // analyze create table command - var cacheMode = CacheType.NONE - var shouldReset = false // These are set when parsing the command statement AST for a CREATE TABLE. - var isCTAS = false var isPartitioned = false if (ast.getToken().getType() == HiveParser.TOK_CREATETABLE) { super.analyzeInternal(ast) - for (ch <- ast.getChildren) { - ch.asInstanceOf[ASTNode].getToken.getType match { - case HiveParser.TOK_QUERY => { - isCTAS = true - // Set the child ASTNode, which will be passed as an argument to - // SemanticAnalzyer#doPhase1(). - child = ch.asInstanceOf[ASTNode] - } - case HiveParser.TOK_TABLEPARTCOLS => { - // If the table that will be created is Hive-partitioned and should be cached, then - // metadata will be initialized in the initializeCachedTableMetadata() call below. - isPartitionedTable = true - // Get the partitioning columns. In Hive, CREATE TABLE ... [PARTITIONED BY] ... - // is handled by a DDLTask (created by the Hive SemanticAnalyzer's genMapRedTasks and - // not to be confused with the Hive DDLSemanticAnalyzer, which handles ALTER/DROP table - // (among other things). Since creating tables in Shark doesn't involve too much - // overhead (e.g. we don't support indexing), just update the Shark - // MemoryMetaDataManager in this method (during the semantic analysis phase). - // TODO(harvey): a Shark-specific DDLTask might be needed once indexing is supported. - partitionColumns = BaseSemanticAnalyzer.getColumns((ASTNode) child.getChild(0), false); - } - case _ => - Unit - } - } - - // The table descriptor can be NULL if the command is a ... - // 1) syntactically valid CREATE TABLE statement. The table specified may or may not already - // exist. If the table already exists, then an exception is thrown by the DDLTask that's - // executed after semantic analysis. - // 2) valid CTAS statement with an IF NOT EXISTS condition, and the specified table already - // exists. If the table to-be-created already exists, and the CTAS statement does not - // have an IF NOT EXISTS condition, then an exception will be thrown by the parent - // SemanticAnalzyer's analyzeInternal() call above. - val createTableDesc = getParseContext.getQB.getTableDesc - - if (!isCTAS || createTableDesc == null) { - return - } else { - val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) - // Note: the CreateTableDesc's table properties are Java Maps, but the TableDesc's table - // properties, which are used during execution, are Java Properties. - val createTableProperties: JavaMap[String, String] = createTableDesc.getTblProps() - - // There are two cases that will enable caching: - // 1) Table name includes "_cached" or "_tachyon". - // 2) The "shark.cache" table property is "true", or the string representation of a supported - // cache mode (heap, Tachyon). - cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) - // Continue planning based on the 'cacheMode' read. - if (cacheMode == CacheType.HEAP || - (createTableDesc.getTableName.endsWith("_cached") && checkTableName)) { - cacheMode = CacheType.HEAP - createTableProperties.put("shark.cache", cacheMode.toString) - } else if (cacheMode == CacheType.TACHYON || - (createTableDesc.getTableName.endsWith("_tachyon") && checkTableName)) { - cacheMode = CacheType.TACHYON - createTableProperties.put("shark.cache", cacheMode.toString) + // Do post-Hive analysis of the CREATE TABLE (e.g detect caching mode). + analyzeCreateTable(ast, qb) match { + case Some(selectStmtASTNode) => { + child = selectStmtASTNode } - - if (CacheType.shouldCache(cacheMode)) { - createTableDesc.setSerName(classOf[ColumnarSerDe].getName) + case None => { + // Done with semantic analysis if the CREATE TABLE statement isn't a CTAS. + return } - - qb.setTableDesc(createTableDesc) - shouldReset = true } } else { SessionState.get().setCommandType(HiveOperation.QUERY) } + // Invariant: At this point, the command includes a query - it's ASTNode has a + // HiveParser.TOK_QUERY somewhere). + // Delegate create view and analyze to Hive. val astTokenType = ast.getToken().getType() if (astTokenType == HiveParser.TOK_CREATEVIEW || astTokenType == HiveParser.TOK_ANALYZE) { @@ -164,6 +108,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } // Continue analyzing from the child ASTNode. + // TODO(harvey): Look into whether doPhase1() can be skipped for CTAS. The + // 'super.analyzeInternal()' call above already calls it. if (!doPhase1(child, qb, initPhase1Ctx())) { return } @@ -176,7 +122,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with logInfo("Completed getting MetaData in Shark Semantic Analysis") // Reset makes sure we don't run the mapred jobs generated by Hive. - if (shouldReset) reset() + reset() // Save the result schema derived from the sink operator produced // by genPlan. This has the correct column names, which clients @@ -205,6 +151,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // TODO: clean the following code. It's too messy to understand... val terminalOpSeq = { if (qb.getParseInfo.isInsertToTable && !qb.isCTAS) { + // Handle an INSERT into a cached table. + // TODO(harvey): Detect the RDD's cache mode. It doesn't make sense to change the cache mode + // using an INSERT... hiveSinkOps.map { hiveSinkOp => val tableName = hiveSinkOp.asInstanceOf[HiveFileSinkOperator].getConf().getTableInfo() .getTableName() @@ -226,7 +175,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with cachedTableName, storageLevel, _resSchema.size, // numColumns - cacheMode == CacheType.TACHYON, // use tachyon + qb.getCacheMode == CacheType.TACHYON, // use tachyon useUnionRDD) } else { throw new SemanticException( @@ -241,7 +190,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // For a single output, we have the option of choosing the output // destination (e.g. CTAS with table property "shark.cache" = "true"). Seq { - if (qb.isCTAS && qb.getTableDesc != null && CacheType.shouldCache(cacheMode)) { + if (qb.isCTAS && qb.getTableDesc != null && CacheType.shouldCache(qb.getCacheMode())) { val storageLevel = MemoryMetadataManager.getStorageLevelFromString( qb.getTableDesc().getTblProps.get("shark.cache.storageLevel")) qb.getTableDesc().getTblProps().put(CachedTableRecovery.QUERY_STRING, ctx.getCmd()) @@ -250,7 +199,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.getTableDesc.getTableName, storageLevel, _resSchema.size, // numColumns - cacheMode == CacheType.TACHYON, // use tachyon + qb.getCacheMode == CacheType.TACHYON, // use tachyon false) } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) @@ -379,6 +328,89 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with rootTasks.head.addDependentTask(crtTblTask) } } + + + def analyzeCreateTable(rootAST: ASTNode, queryBlock: QueryBlock): Option[ASTNode] = { + // If we detect that the CREATE TABLE is part of a CTAS, then this is set to the root node of the + // SELECT statement + var selectStmtASTNode: Option[ASTNode] = None + + // TODO(harvey): Probably don't need this. We might be able to reuse the QB passed into this + // method, as long as it was created from getParseContext.getQB after the + // super.analyzeInternal() call. + // That QB's createTableDesc should have everything needed (isCTAS(), partCols...). + var isCTAS = false + var isPartitionedTable = false + + for (ch <- rootAST.getChildren) { + ch.asInstanceOf[ASTNode].getToken.getType match { + case HiveParser.TOK_QUERY => { + isCTAS = true + selectStmtASTNode = Some(ch.asInstanceOf[ASTNode]) + } + case HiveParser.TOK_TABLEPARTCOLS => { + // If the table that will be created is Hive-partitioned and should be cached, then + // metadata will be initialized in the initializeCachedTableMetadata() call below. + isPartitionedTable = true + // Get the partitioning columns. In Hive, a 'CREATE TABLE ... [PARTITIONED BY] ...' + // command is handled by a DDLTask (created by the Hive SemanticAnalyzer's genMapRedTasks + // and not to be confused with the Hive DDLSemanticAnalyzer, which handles ALTER/DROP + // table (among other things). Since creating tables in Shark doesn't involve too much + // overhead (e.g. we don't support indexing), just directly update the Shark + // MemoryMetaDataManager in this block. + // TODO(harvey): A Shark-specific DDLTask (specifying a Spark job) might be needed once + // indexing is supported. + + // partitionColumns = BaseSemanticAnalyzer.getColumnNames( + // rootAST.getChild(0).asInstanceOf[ASTNode]) + } + case _ => Unit + } + } + + // The 'createTableDesc' can be NULL if the command is a ... + // 1) syntactically valid CREATE TABLE statement. The table specified may or may not already + // exist. If the table already exists, then an exception is thrown by the DDLTask that's + // executed after semantic analysis. + // 2) valid CTAS statement with an IF NOT EXISTS condition, and the specified table already + // exists. If the table to-be-created already exists, and the CTAS statement does not + // have an IF NOT EXISTS condition, then an exception will be thrown by + // SemanticAnalzyer#analyzeInternal(). + val createTableDesc = getParseContext.getQB.getTableDesc + if (!isCTAS || createTableDesc == null) { + return selectStmtASTNode + } else { + val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) + // Note: the CreateTableDesc's table properties are Java Maps, but the TableDesc's table + // properties, which are used during execution, are Java Properties. + val createTableProperties: JavaMap[String, String] = createTableDesc.getTblProps() + + // There are two cases that will enable caching: + // 1) Table name includes "_cached" or "_tachyon". + // 2) The "shark.cache" table property is "true", or the string representation of a supported + // cache mode (heap, Tachyon). + var cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) + // Continue planning based on the 'cacheMode' read. + if (cacheMode == CacheType.HEAP || + (createTableDesc.getTableName.endsWith("_cached") && checkTableName)) { + cacheMode = CacheType.HEAP + createTableProperties.put("shark.cache", cacheMode.toString) + } else if (cacheMode == CacheType.TACHYON || + (createTableDesc.getTableName.endsWith("_tachyon") && checkTableName)) { + cacheMode = CacheType.TACHYON + createTableProperties.put("shark.cache", cacheMode.toString) + } + + if (CacheType.shouldCache(cacheMode)) { + createTableDesc.setSerName(classOf[ColumnarSerDe].getName) + } + + queryBlock.setCacheMode(cacheMode) + queryBlock.setTableDesc(createTableDesc) + } + + return selectStmtASTNode + } } From cba56210c70ea1da46b65d7ba3eda24f9e930beb Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 19:21:40 -0700 Subject: [PATCH 020/331] Return union of all Hive-partition RDDs for SharkEnv#unpersist() and MemoryMetadataManager#unpersist(). Also added Option[RDD[_]] in MemoryTable to represent an empty table. --- src/main/scala/shark/SharkEnv.scala | 5 +- .../memstore2/MemoryMetadataManager.scala | 50 ++++++++++--------- .../scala/shark/memstore2/MemoryTable.scala | 6 +-- src/main/scala/shark/parse/QueryBlock.scala | 4 +- .../shark/parse/SharkSemanticAnalyzer.scala | 36 ++++++------- 5 files changed, 49 insertions(+), 52 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index f082fee4..3a5903f8 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -114,7 +114,7 @@ object SharkEnv extends LogHelper { val addedFiles = HashSet[String]() val addedJars = HashSet[String]() - def unpersist(key: String) { + def unpersist(key: String): Option[RDD[_]] = { if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(key)) { if (SharkEnv.tachyonUtil.dropTable(key)) { logInfo("Table " + key + " was deleted from Tachyon."); @@ -122,8 +122,7 @@ object SharkEnv extends LogHelper { logWarning("Failed to remove table " + key + " from Tachyon."); } } - - memoryMetadataManager.unpersist(key) + return memoryMetadataManager.unpersist(key) } /** Cleans up and shuts down the Shark environments. */ diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 75d190e4..145003e8 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -33,35 +33,31 @@ class MemoryMetadataManager { private val _keyToMemoryTable: ConcurrentMap[String, MemoryTable] = new ConcurrentHashMap[String, MemoryTable]() - // TODO(harvey): Support stats for Hive-partitioned tables. + // TODO(harvey): Support stats for cached Hive-partitioned tables. private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) + def add(key: String, isHivePartitioned: Boolean) { + _keyToMemoryTable.put(key, new MemoryTable(key, isHivePartitioned)) + } + def put(key: String, rdd: RDD[_]) { - if (!contains(key)) { - _keyToMemoryTable(key.toLowerCase) = new MemoryTable(key, false /* isHivePartitioned */) - } _keyToMemoryTable(key.toLowerCase).tableRDD = rdd } def putHivePartition(key: String, partitionColumn: String, rdd: RDD[_]) { - if (!contains(key)) { - _keyToMemoryTable(key.toLowerCase) = new MemoryTable(key, true /* isHivePartitioned */) - } _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumn) = rdd } def get(key: String): Option[RDD[_]] = { - _keyToMemoryTable.get(key.toLowerCase) match { - case Some(memoryTable) => return Some(memoryTable.tableRDD) - case None => return None - } + val memoryTableValue: Option[MemoryTable] = _keyToMemoryTable.get(key.toLowerCase) + return memoryTableValue.flatMap(_.tableRDD) } - def getHivePartition(key: String, partitionColumn: String): RDD[_] = { - return _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumn) + def getHivePartition(key: String, partitionColumn: String): Option[RDD[_]] = { + return _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs.get(partitionColumn) } def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { @@ -85,8 +81,11 @@ class MemoryMetadataManager { * as well. * * @param key Name of the table to drop. + * @return Option::isEmpty() is true of there is no MemoryTable corresponding to 'key' in + * _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will be + * a UnionRDD comprising all RDDs for all Hive-partitions. */ - def unpersist(key: String) { + def unpersist(key: String): Option[RDD[_]] = { def unpersistRDD(rdd: RDD[_]) { rdd match { case u: UnionRDD[_] => { @@ -99,23 +98,28 @@ class MemoryMetadataManager { case r => r.unpersist() } } - def unpersistMemoryTable(memoryTable: MemoryTable) { + def unpersistMemoryTable(memoryTable: MemoryTable): Option[RDD[_]] = { if (memoryTable.isHivePartitioned) { // unpersist() all RDDs for all Hive-partitions. - memoryTable.hivePartitionRDDs.mapValues(unpersistRDD(_)) + val hivePartitionRDDs = + memoryTable.hivePartitionRDDs.values.toSeq.asInstanceOf[Seq[RDD[Any]]] + if (hivePartitionRDDs.size > 0) { + return Some(new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs)) + } + return None } else { - unpersistRDD(memoryTable.tableRDD) + if (memoryTable.tableRDD.isDefined) { + unpersistRDD(memoryTable.tableRDD.get) + } + return memoryTable.tableRDD } } + // Remove MemoryTable's entry from Shark metadata. - val memoryTableValue = _keyToMemoryTable.remove(key.toLowerCase()) _keyToStats.remove(key) - // Unpersist the MemoryTable using the nested helper functions above. - memoryTableValue match { - case Some(memoryTable) => unpersistMemoryTable(memoryTable) - case None => Unit - } + val memoryTableValue: Option[MemoryTable] = _keyToMemoryTable.remove(key.toLowerCase) + return memoryTableValue.flatMap(unpersistMemoryTable(_)) } } diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index b2e0206d..6d441002 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -40,7 +40,7 @@ import org.apache.spark.rdd.RDD private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: Boolean) { /** Should only be used if the table is not Hive-partitioned. */ - private var _tableRDD: RDD[_] = _ + private var _tableRDD: Option[RDD[_]] = None /** * Should only be used if a cached table is Hive-partitioned. @@ -48,7 +48,7 @@ private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: B private val _hivePartitionRDDs: Map[String, RDD[_]] = if (isHivePartitioned) { new JavaHashMap[String, RDD[_]]() } else { null } - def tableRDD: RDD[_] = { + def tableRDD: Option[RDD[_]] = { assert ( !isHivePartitioned, "Table " + tableName + " is Hive-partitioned. Use MemoryTableDesc::hivePartitionRDDs() " + @@ -63,7 +63,7 @@ private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: B "Table " + tableName + " is Hive-partitioned. Pass in a map of pairs " + "the 'hivePartitionRDDs =' setter." ) - _tableRDD = value + _tableRDD = Some(value) } def hivePartitionRDDs: Map[String, RDD[_]] = { diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 09d4fb9a..8e3fad38 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2012 The Regents of The University California. + * Copyright (C) 2012 The Regents of The University California. * All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -25,7 +25,7 @@ import shark.memstore2.CacheType._ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) extends HiveQueryBlock(outerID, alias, isSubQuery) { - // The CacheType for the table of a CREATE TABLE/CTAS, + // The CacheType for the table that will be created from CREATE TABLE/CTAS. private var cacheMode = CacheType.NONE def setCacheMode(cacheModeToSet: CacheType) { diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index eacb9690..f799aaff 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -174,8 +174,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with hiveSinkOp, cachedTableName, storageLevel, - _resSchema.size, // numColumns - qb.getCacheMode == CacheType.TACHYON, // use tachyon + _resSchema.size, // numColumns + qb.getCacheMode == CacheType.TACHYON, // use tachyon useUnionRDD) } else { throw new SemanticException( @@ -337,10 +337,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // TODO(harvey): Probably don't need this. We might be able to reuse the QB passed into this // method, as long as it was created from getParseContext.getQB after the - // super.analyzeInternal() call. - // That QB's createTableDesc should have everything needed (isCTAS(), partCols...). + // super.analyzeInternal() call. That QB's createTableDesc should have everything + // needed (e.g. isCTAS(), partCols). var isCTAS = false - var isPartitionedTable = false + var isHivePartitioned = false for (ch <- rootAST.getChildren) { ch.asInstanceOf[ASTNode].getToken.getType match { @@ -349,20 +349,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with selectStmtASTNode = Some(ch.asInstanceOf[ASTNode]) } case HiveParser.TOK_TABLEPARTCOLS => { - // If the table that will be created is Hive-partitioned and should be cached, then - // metadata will be initialized in the initializeCachedTableMetadata() call below. - isPartitionedTable = true - // Get the partitioning columns. In Hive, a 'CREATE TABLE ... [PARTITIONED BY] ...' - // command is handled by a DDLTask (created by the Hive SemanticAnalyzer's genMapRedTasks - // and not to be confused with the Hive DDLSemanticAnalyzer, which handles ALTER/DROP - // table (among other things). Since creating tables in Shark doesn't involve too much - // overhead (e.g. we don't support indexing), just directly update the Shark - // MemoryMetaDataManager in this block. - // TODO(harvey): A Shark-specific DDLTask (specifying a Spark job) might be needed once - // indexing is supported. - - // partitionColumns = BaseSemanticAnalyzer.getColumnNames( - // rootAST.getChild(0).asInstanceOf[ASTNode]) + isHivePartitioned = true } case _ => Unit } @@ -377,6 +364,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // have an IF NOT EXISTS condition, then an exception will be thrown by // SemanticAnalzyer#analyzeInternal(). val createTableDesc = getParseContext.getQB.getTableDesc + val tableName = createTableDesc.getTableName if (!isCTAS || createTableDesc == null) { return selectStmtASTNode } else { @@ -392,11 +380,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with var cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) // Continue planning based on the 'cacheMode' read. if (cacheMode == CacheType.HEAP || - (createTableDesc.getTableName.endsWith("_cached") && checkTableName)) { + (checkTableName && tableName.endsWith("_cached"))) { cacheMode = CacheType.HEAP createTableProperties.put("shark.cache", cacheMode.toString) } else if (cacheMode == CacheType.TACHYON || - (createTableDesc.getTableName.endsWith("_tachyon") && checkTableName)) { + (checkTableName && tableName.endsWith("_tachyon"))) { cacheMode = CacheType.TACHYON createTableProperties.put("shark.cache", cacheMode.toString) } @@ -409,6 +397,12 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with queryBlock.setTableDesc(createTableDesc) } + // In Hive, a CREATE TABLE command is handled by a DDLTask, which in this case, is created by + // the Hive SemanticAnalyzer's genMapRedTasks and not Hive's DDLSemanticAnalyzer. Since + // creating tables in Shark doesn't involve too much overhead (we don't support features such + // as indexing), just directly update the Shark MemoryMetaDataManager in this method. + MemoryMetadataManager.add(tableName, isHivePartitioned) + return selectStmtASTNode } } From 263004db9e54bd48fb483a0c5901ae171a5efb53 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 21:04:20 -0700 Subject: [PATCH 021/331] Header comment for QueryBlock. --- src/main/scala/shark/parse/QueryBlock.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 8e3fad38..f729f0c9 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -22,6 +22,11 @@ import org.apache.hadoop.hive.ql.parse.{QB => HiveQueryBlock} import shark.memstore2.CacheType import shark.memstore2.CacheType._ + +/** + * A container for flags and table metadata that's used in SharkSemanticAnalyzer while parsing + * and analyzing command ASTs (e.g. in analyzeCreateTable()). + */ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) extends HiveQueryBlock(outerID, alias, isSubQuery) { From dcb2dac5a0f58ac80ec1180f303dafb82e410532 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 21:04:39 -0700 Subject: [PATCH 022/331] Disable cached, Hive-partitioned tables for now. --- .../shark/parse/SharkSemanticAnalyzer.scala | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index f799aaff..efdc3244 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -379,8 +379,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // cache mode (heap, Tachyon). var cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) // Continue planning based on the 'cacheMode' read. - if (cacheMode == CacheType.HEAP || - (checkTableName && tableName.endsWith("_cached"))) { + if (cacheMode == CacheType.HEAP || (checkTableName && tableName.endsWith("_cached"))) { cacheMode = CacheType.HEAP createTableProperties.put("shark.cache", cacheMode.toString) } else if (cacheMode == CacheType.TACHYON || @@ -391,18 +390,24 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with if (CacheType.shouldCache(cacheMode)) { createTableDesc.setSerName(classOf[ColumnarSerDe].getName) + + // TODO(harvey): Remove once it's supported ... + if (isHivePartitioned) { + throw new SemanticException( + "Support for cached, Hive-partitioned tables coming soon!") + } + + // In Hive, a CREATE TABLE command is handled by a DDLTask, which in this case, is created + // by the Hive SemanticAnalyzer's genMapRedTasks and not Hive's DDLSemanticAnalyzer. Since + // creating tables in Shark doesn't involve too much overhead (we don't support features + // such as indexing), just directly update the Shark MemoryMetaDataManager in this method. + MemoryMetadataManager.add(tableName, isHivePartitioned) } queryBlock.setCacheMode(cacheMode) queryBlock.setTableDesc(createTableDesc) } - // In Hive, a CREATE TABLE command is handled by a DDLTask, which in this case, is created by - // the Hive SemanticAnalyzer's genMapRedTasks and not Hive's DDLSemanticAnalyzer. Since - // creating tables in Shark doesn't involve too much overhead (we don't support features such - // as indexing), just directly update the Shark MemoryMetaDataManager in this method. - MemoryMetadataManager.add(tableName, isHivePartitioned) - return selectStmtASTNode } } From 4627ed1cc39c448dcf8c1f73caeb215af6e40bf0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 21:13:53 -0700 Subject: [PATCH 023/331] =?UTF-8?q?Update=20some=20comments=20in=20MemoryT?= =?UTF-8?q?able,=20fix=20small=20SharkSemanticAnalyzer=20compile=20bug?= =?UTF-8?q?=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- src/main/scala/shark/memstore2/MemoryTable.scala | 11 +++++------ .../scala/shark/parse/SharkSemanticAnalyzer.scala | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 6d441002..6a8a8e13 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -30,8 +30,7 @@ import org.apache.spark.rdd.RDD * wrapper around either an RDD or multiple RDDs if the Shark table is Hive-partitioned. * Note that a Hive-partition of a table is different from an RDD partition. Each Hive-partition * is stored as a subdirectory of the table subdirectory in the warehouse directory - * (e.g. /user/hive/warehouse). So, every Hive-Partition is loaded into Shark as an RDD, and is - * cached as one if the user-specifies it. + * (e.g. /user/hive/warehouse). So, every Hive-Partition is loaded into Shark as an RDD. * * TODO(harvey): It could be useful to make MemoryTable a parent class, and have other table types, * such as HivePartitionedTable or TachyonTable, subclass it. For now, there isn't @@ -39,12 +38,12 @@ import org.apache.spark.rdd.RDD */ private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: Boolean) { - /** Should only be used if the table is not Hive-partitioned. */ + // Should only be used if the table is not Hive-partitioned. _tableRDD.isEmpty() is true if the + // table does not contain any data (e.g. it was created from a CREATE TABLE command, but never + // populated with data). private var _tableRDD: Option[RDD[_]] = None - /** - * Should only be used if a cached table is Hive-partitioned. - */ + // Should only be used if a cached table is Hive-partitioned. private val _hivePartitionRDDs: Map[String, RDD[_]] = if (isHivePartitioned) { new JavaHashMap[String, RDD[_]]() } else { null } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index efdc3244..baf0ee28 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -401,7 +401,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // by the Hive SemanticAnalyzer's genMapRedTasks and not Hive's DDLSemanticAnalyzer. Since // creating tables in Shark doesn't involve too much overhead (we don't support features // such as indexing), just directly update the Shark MemoryMetaDataManager in this method. - MemoryMetadataManager.add(tableName, isHivePartitioned) + SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned) } queryBlock.setCacheMode(cacheMode) From d94d3d23ca8be7bdbcbd482cd9348c57b43c9a91 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 22:09:34 -0700 Subject: [PATCH 024/331] Add an isHivePartitioned() fn to MemoryMetadataManager. Helpful for testing. --- .../scala/shark/memstore2/MemoryMetadataManager.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 145003e8..83a5db8e 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -27,7 +27,8 @@ import org.apache.spark.storage.StorageLevel import shark.SharkConfVars - +// TODO(harvey): Re-evaluate the interfaces to this class. For example, add() could be renamed to +// addCreatedTable(). class MemoryMetadataManager { private val _keyToMemoryTable: ConcurrentMap[String, MemoryTable] = @@ -39,6 +40,13 @@ class MemoryMetadataManager { def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) + def isHivePartitioned(key: String): Boolean = { + _keyToMemoryTable.get(key.toLowerCase) match { + case Some(memoryTable) => return memoryTable.isHivePartitioned + case None => return false + } + } + def add(key: String, isHivePartitioned: Boolean) { _keyToMemoryTable.put(key, new MemoryTable(key, isHivePartitioned)) } From b7c1d187e70dc74488bf4d4b7934896197c4eaaf Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 9 Sep 2013 22:10:18 -0700 Subject: [PATCH 025/331] Add TODO for creation of cached tables using non-CTAS CREATE TABLE. --- .../shark/parse/SharkSemanticAnalyzer.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index baf0ee28..63a283c1 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -337,8 +337,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // TODO(harvey): Probably don't need this. We might be able to reuse the QB passed into this // method, as long as it was created from getParseContext.getQB after the - // super.analyzeInternal() call. That QB's createTableDesc should have everything - // needed (e.g. isCTAS(), partCols). + // super.analyzeInternal() call. That QB and the createTableDesc should have + // everything (e.g. isCTAS(), partCols). var isCTAS = false var isHivePartitioned = false @@ -364,10 +364,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // have an IF NOT EXISTS condition, then an exception will be thrown by // SemanticAnalzyer#analyzeInternal(). val createTableDesc = getParseContext.getQB.getTableDesc - val tableName = createTableDesc.getTableName - if (!isCTAS || createTableDesc == null) { - return selectStmtASTNode - } else { + if (isCTAS && createTableDesc != null) { + val tableName = createTableDesc.getTableName val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) // Note: the CreateTableDesc's table properties are Java Maps, but the TableDesc's table // properties, which are used during execution, are Java Properties. @@ -391,8 +389,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with if (CacheType.shouldCache(cacheMode)) { createTableDesc.setSerName(classOf[ColumnarSerDe].getName) - // TODO(harvey): Remove once it's supported ... if (isHivePartitioned) { + // TODO(harvey): Remove once it's supported ... throw new SemanticException( "Support for cached, Hive-partitioned tables coming soon!") } @@ -401,7 +399,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // by the Hive SemanticAnalyzer's genMapRedTasks and not Hive's DDLSemanticAnalyzer. Since // creating tables in Shark doesn't involve too much overhead (we don't support features // such as indexing), just directly update the Shark MemoryMetaDataManager in this method. - SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned) + + // Make sure that the table exists. + // TODO(harvey): This might have to go in a SparkDDLTask wrapper. + // SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned) } queryBlock.setCacheMode(cacheMode) From 0f52ecc53aacadc4f8978ef78d649c2e734d51d8 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 10 Sep 2013 18:32:38 -0700 Subject: [PATCH 026/331] Fix comment about QBs in SharkSemanticAnalyzer. --- src/main/scala/shark/parse/SharkSemanticAnalyzer.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 63a283c1..285a9938 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -336,9 +336,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with var selectStmtASTNode: Option[ASTNode] = None // TODO(harvey): Probably don't need this. We might be able to reuse the QB passed into this - // method, as long as it was created from getParseContext.getQB after the - // super.analyzeInternal() call. That QB and the createTableDesc should have - // everything (e.g. isCTAS(), partCols). + // method, as long as it was created after the super.analyzeInternal() call. + // That QB and the createTableDesc should have everything (e.g. isCTAS(), + // partCols). Note that the QB might not be accessible from getParseContext(), + // since the SemanticAnalyzer#analyzeInternal() doesn't set (this.qb = qb) for a + // non-CTAS. var isCTAS = false var isHivePartitioned = false From fd1dc4f53e0c8521f750295178f4ce0690de786a Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 10 Sep 2013 18:44:23 -0700 Subject: [PATCH 027/331] Better variable name for QueryBlock's cache mode. --- src/main/scala/shark/parse/QueryBlock.scala | 8 ++++---- .../scala/shark/parse/SharkSemanticAnalyzer.scala | 14 ++++++++------ 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index f729f0c9..9f78fec8 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -31,11 +31,11 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) extends HiveQueryBlock(outerID, alias, isSubQuery) { // The CacheType for the table that will be created from CREATE TABLE/CTAS. - private var cacheMode = CacheType.NONE + private var cacheModeForCreateTable = CacheType.NONE - def setCacheMode(cacheModeToSet: CacheType) { - cacheMode = cacheModeToSet + def setCacheModeForCreateTable(cacheModeToSet: CacheType) { + cacheModeForCreateTable = cacheModeToSet } - def getCacheMode(): CacheType = cacheMode + def getCacheModeForCreateTable(): CacheType = cacheModeForCreateTable } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 285a9938..6cbc2020 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -152,8 +152,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val terminalOpSeq = { if (qb.getParseInfo.isInsertToTable && !qb.isCTAS) { // Handle an INSERT into a cached table. - // TODO(harvey): Detect the RDD's cache mode. It doesn't make sense to change the cache mode - // using an INSERT... + // TODO(harvey): Detect the RDD's cache mode. This could be easily be done by using the + // cachedTableName below. The cache mode property could be added to + // MemoryTable. hiveSinkOps.map { hiveSinkOp => val tableName = hiveSinkOp.asInstanceOf[HiveFileSinkOperator].getConf().getTableInfo() .getTableName() @@ -175,7 +176,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with cachedTableName, storageLevel, _resSchema.size, // numColumns - qb.getCacheMode == CacheType.TACHYON, // use tachyon + qb.getCacheModeForCreateTable == CacheType.TACHYON, // use tachyon useUnionRDD) } else { throw new SemanticException( @@ -190,7 +191,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // For a single output, we have the option of choosing the output // destination (e.g. CTAS with table property "shark.cache" = "true"). Seq { - if (qb.isCTAS && qb.getTableDesc != null && CacheType.shouldCache(qb.getCacheMode())) { + if (qb.isCTAS && qb.getTableDesc != null && + CacheType.shouldCache(qb.getCacheModeForCreateTable())) { val storageLevel = MemoryMetadataManager.getStorageLevelFromString( qb.getTableDesc().getTblProps.get("shark.cache.storageLevel")) qb.getTableDesc().getTblProps().put(CachedTableRecovery.QUERY_STRING, ctx.getCmd()) @@ -199,7 +201,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.getTableDesc.getTableName, storageLevel, _resSchema.size, // numColumns - qb.getCacheMode == CacheType.TACHYON, // use tachyon + qb.getCacheModeForCreateTable == CacheType.TACHYON, // use tachyon false) } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) @@ -407,7 +409,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned) } - queryBlock.setCacheMode(cacheMode) + queryBlock.setCacheModeForCreateTable(cacheMode) queryBlock.setTableDesc(createTableDesc) } From 5adbdd0187ca9fc48a5a31e9605a696b586abc2d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 10 Sep 2013 18:52:42 -0700 Subject: [PATCH 028/331] Don't error out in put() and putHivePartition() for now. --- .../scala/shark/memstore2/MemoryMetadataManager.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 83a5db8e..fc58b6bd 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -52,10 +52,20 @@ class MemoryMetadataManager { } def put(key: String, rdd: RDD[_]) { + if (!_keyToMemoryTable.contains(key.toLowerCase)) { + // TODO(harvey): Remove this once CREATE TABLE/CTAS handling involves calling add(). For now, + // CTAS is done in the MemoryStoreSinkOperator, which also "adds" the RDD for + // the created table to MemoryMetadataManager. + add(key, false /* isHivePartitioned */) + } _keyToMemoryTable(key.toLowerCase).tableRDD = rdd } def putHivePartition(key: String, partitionColumn: String, rdd: RDD[_]) { + if (!_keyToMemoryTable.contains(key.toLowerCase)) { + // TODO(harvey): See comment for put() above. + add(key, true /* isHivePartitioned */) + } _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumn) = rdd } From 797a628ac4cf40d281a6f756a0977ce59af07057 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 10 Sep 2013 19:02:26 -0700 Subject: [PATCH 029/331] Table names added in MemoryMetadatamanager should be case-insensitive.... --- src/main/scala/shark/memstore2/MemoryMetadataManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index fc58b6bd..ce3f52a8 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -48,7 +48,7 @@ class MemoryMetadataManager { } def add(key: String, isHivePartitioned: Boolean) { - _keyToMemoryTable.put(key, new MemoryTable(key, isHivePartitioned)) + _keyToMemoryTable.put(key.toLowerCase, new MemoryTable(key, isHivePartitioned)) } def put(key: String, rdd: RDD[_]) { From 5c542394a4c93e815c5f36f967026264dff81e90 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 11 Sep 2013 13:23:52 +0800 Subject: [PATCH 030/331] Fix name for non assembly jar. --- project/SharkBuild.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 5ea98436..8eeca055 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -118,7 +118,6 @@ object SharkBuild extends Build { ) def assemblyProjSettings = Seq( - name := "shark-assembly", jarName in assembly <<= version map { v => "shark-assembly-" + v + "-hadoop" + HADOOP_VERSION + ".jar" } ) ++ assemblySettings ++ extraAssemblySettings From 8401ed34d94b3baa3cb88c2b1e4c11a08a529b4a Mon Sep 17 00:00:00 2001 From: Alexander Pivovarov Date: Thu, 12 Sep 2013 01:07:41 -0700 Subject: [PATCH 031/331] sbt - check HIVE_DEV_HOME for eclipse --- sbt/sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbt/sbt b/sbt/sbt index e4b1c072..d6b1f360 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,9 +5,9 @@ if [ -e $SHARK_CONF_DIR/shark-env.sh ] ; then . $SHARK_CONF_DIR/shark-env.sh fi -if [[ "$@" == *"test"* ]]; then +if [[ "$@" == *"test"* ]] || [ "$@" == "eclipse" ]; then if [ "x$HIVE_DEV_HOME" == "x" ]; then - echo "No HIVE_DEV_HOME specified. Required for tests. Please set HIVE_DEV_HOME." + echo "No HIVE_DEV_HOME specified. Required for tests and eclipse. Please set HIVE_DEV_HOME." exit 1 fi fi From ae1abbab61d3ddd63a2af786515a43804b91ef49 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 12 Sep 2013 01:46:56 -0700 Subject: [PATCH 032/331] More comments and minor refactoring of SharkSemanticAnalyzer. --- .../shark/parse/SharkSemanticAnalyzer.scala | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 6cbc2020..ec94d3f9 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -77,20 +77,30 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with logInfo("Starting Shark Semantic Analysis") //TODO: can probably reuse Hive code for this - // analyze create table command + var shouldReset = false - // These are set when parsing the command statement AST for a CREATE TABLE. - var isPartitioned = false - - if (ast.getToken().getType() == HiveParser.TOK_CREATETABLE) { + val astTokenType = ast.getToken().getType() + if (astTokenType == HiveParser.TOK_CREATEVIEW || astTokenType == HiveParser.TOK_ANALYZE) { + // Delegate create view and analyze to Hive. + super.analyzeInternal(ast) + return + } else if (astTokenType == HiveParser.TOK_CREATETABLE) { + // Use Hive to do a first analysis pass. super.analyzeInternal(ast) // Do post-Hive analysis of the CREATE TABLE (e.g detect caching mode). analyzeCreateTable(ast, qb) match { case Some(selectStmtASTNode) => { + // The 'child' is set to the SELECT statement root node, with is a HiveParer.HIVE_QUERY. child = selectStmtASTNode + // Hive will generate MapReduce tasks for the SELECT. Avoid executing those tasks by + // reset()-ing some Hive SemanticAnalyzer state after phase 1 of analysis below. + // TODO(harvey): This might be too much. SharkSemanticAnalyzer could directly clear + // 'rootTasks', since it's a protected field. + shouldReset = true } case None => { - // Done with semantic analysis if the CREATE TABLE statement isn't a CTAS. + // Done with semantic analysis if the CREATE TABLE statement isn't a CTAS. The DDLTask + // created from 'super.analyzeInternal()' will be used to create the table. return } } @@ -101,12 +111,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // Invariant: At this point, the command includes a query - it's ASTNode has a // HiveParser.TOK_QUERY somewhere). - // Delegate create view and analyze to Hive. - val astTokenType = ast.getToken().getType() - if (astTokenType == HiveParser.TOK_CREATEVIEW || astTokenType == HiveParser.TOK_ANALYZE) { - return super.analyzeInternal(ast) - } - // Continue analyzing from the child ASTNode. // TODO(harvey): Look into whether doPhase1() can be skipped for CTAS. The // 'super.analyzeInternal()' call above already calls it. @@ -122,7 +126,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with logInfo("Completed getting MetaData in Shark Semantic Analysis") // Reset makes sure we don't run the mapred jobs generated by Hive. - reset() + if (shouldReset) reset() // Save the result schema derived from the sink operator produced // by genPlan. This has the correct column names, which clients From 5937aa692ef159bf7238462db5c6f0ef23246f39 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 13 Sep 2013 10:39:56 +0800 Subject: [PATCH 033/331] Exclude asm, jackson, and netty from hadoop client. Changed jets3t's version to 0.7.1. --- project/SharkBuild.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 5ea98436..b1374716 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -45,6 +45,9 @@ object SharkBuild extends Build { val excludeKyro = ExclusionRule(organization = "de.javakaffee") val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") + val excludeAsm = ExclusionRule(organization = "asm") + val excludeSnappy = ExclusionRule(organization = "org.xerial.snappy") def coreSettings = Defaults.defaultSettings ++ Seq( @@ -100,7 +103,7 @@ object SharkBuild extends Build { "org.apache.spark" %% "spark-core" % SPARK_VERSION, "org.apache.spark" %% "spark-repl" % SPARK_VERSION, "com.google.guava" % "guava" % "14.0.1", - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeNetty), + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), // See https://code.google.com/p/guava-libraries/issues/detail?id=1095 "com.google.code.findbugs" % "jsr305" % "1.3.+", @@ -112,7 +115,7 @@ object SharkBuild extends Build { // Test infrastructure "org.scalatest" %% "scalatest" % "1.9.1" % "test", "junit" % "junit" % "4.10" % "test", - "net.java.dev.jets3t" % "jets3t" % "0.9.0", + "net.java.dev.jets3t" % "jets3t" % "0.7.1", "com.novocode" % "junit-interface" % "0.8" % "test") ++ (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop) ) else None).toSeq ) From 4b655e99388cddc948752a1a63cc289b9836fdf7 Mon Sep 17 00:00:00 2001 From: Sarah Gerweck Date: Thu, 12 Sep 2013 13:13:46 -0700 Subject: [PATCH 034/331] Pull the Hadoop version from the environment. This introduces two environment variables that you can use to control the version of Hadoop that you build against. The Spark project uses `SPARK_HADOOP_VERSION`, which is supported. By analogy, we also support `SHARK_HADOOP_VERSION`, which has precedence over `SPARK_HADOOP_VERSION`. If neither of these environment variables are set at build time, the defaults from the build script are used. The algorithm looks like this: 1. Use `SHARK_HADOOP_VERSION` if set. 2. Use `SPARK_HADOOP_VERSION` if set. 3. Use the default that is hard-coded into the build definition. --- project/SharkBuild.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index bd90eb93..ec6b6270 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -32,7 +32,13 @@ object SharkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.1" for Apache releases, or "0.20.2-cdh3u3" for Cloudera Hadoop. - val HADOOP_VERSION = "1.0.4" + val DEFAULT_HADOOP_VERSION = "1.0.4" + + import scala.util.Properties.{ envOrNone => env } + lazy val hadoopVersion = env("SHARK_HADOOP_VERSION") orElse + env("SPARK_HADOOP_VERSION") getOrElse + DEFAULT_HADOOP_VERSION + // Whether to build Shark with Tachyon jar. val TACHYON_ENABLED = false @@ -103,7 +109,7 @@ object SharkBuild extends Build { "org.apache.spark" %% "spark-core" % SPARK_VERSION, "org.apache.spark" %% "spark-repl" % SPARK_VERSION, "com.google.guava" % "guava" % "14.0.1", - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), // See https://code.google.com/p/guava-libraries/issues/detail?id=1095 "com.google.code.findbugs" % "jsr305" % "1.3.+", @@ -121,7 +127,7 @@ object SharkBuild extends Build { ) def assemblyProjSettings = Seq( - jarName in assembly <<= version map { v => "shark-assembly-" + v + "-hadoop" + HADOOP_VERSION + ".jar" } + jarName in assembly <<= version map { v => "shark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" } ) ++ assemblySettings ++ extraAssemblySettings def extraAssemblySettings() = Seq( From 5f84acc155c2c6ee40c6657e92de865a8ca0bbda Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 18 Sep 2013 09:11:39 +0800 Subject: [PATCH 035/331] Add Spark-yarn dependency when SHARK_YARN=true, which by default is false. Also fix Hadoop FileSystem Sevice Meta file assembly rules. --- project/SharkBuild.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index bd90eb93..fa6deb68 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -34,6 +34,12 @@ object SharkBuild extends Build { // "1.0.1" for Apache releases, or "0.20.2-cdh3u3" for Cloudera Hadoop. val HADOOP_VERSION = "1.0.4" + // Whether to build Shark with Yarn support + val YARN_ENABLED = scala.util.Properties.envOrNone("SHARK_YARN") match { + case None => false + case Some(v) => v.toBoolean + } + // Whether to build Shark with Tachyon jar. val TACHYON_ENABLED = false @@ -117,6 +123,7 @@ object SharkBuild extends Build { "junit" % "junit" % "4.10" % "test", "net.java.dev.jets3t" % "jets3t" % "0.7.1", "com.novocode" % "junit-interface" % "0.8" % "test") ++ + (if (YARN_ENABLED) Some("org.apache.spark" %% "spark-yarn" % SPARK_VERSION) else None).toSeq ++ (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop) ) else None).toSeq ) @@ -129,6 +136,7 @@ object SharkBuild extends Build { mergeStrategy in assembly := { case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard + case "META-INF/services/org.apache.hadoop.fs.FileSystem" => MergeStrategy.concat case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first } From 473251825691f40dd93d2efee84d7eb027f5d955 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 17 Sep 2013 21:33:04 -0700 Subject: [PATCH 036/331] Fix a check in sbt/sbt (if there are multiple command line args, sbt/sbt throws an error in line 8). --- sbt/sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbt/sbt b/sbt/sbt index d6b1f360..21441ab8 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,8 +5,8 @@ if [ -e $SHARK_CONF_DIR/shark-env.sh ] ; then . $SHARK_CONF_DIR/shark-env.sh fi -if [[ "$@" == *"test"* ]] || [ "$@" == "eclipse" ]; then - if [ "x$HIVE_DEV_HOME" == "x" ]; then +if [[ "$@" == *"test"* ]] || [[ "$@" == "eclipse" ]]; then + if [[ "x$HIVE_DEV_HOME" == "x" ]]; then echo "No HIVE_DEV_HOME specified. Required for tests and eclipse. Please set HIVE_DEV_HOME." exit 1 fi From 0e650627b2160ddfbda2a547de006169c5fcc049 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 17 Sep 2013 21:33:27 -0700 Subject: [PATCH 037/331] Use env shortcut in build file for YARN build. --- project/SharkBuild.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 3e0825d5..088e8fc0 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -21,6 +21,8 @@ import Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ +import scala.util.Properties.{ envOrNone => env } + object SharkBuild extends Build { // Shark version @@ -34,17 +36,12 @@ object SharkBuild extends Build { // "1.0.1" for Apache releases, or "0.20.2-cdh3u3" for Cloudera Hadoop. val DEFAULT_HADOOP_VERSION = "1.0.4" - import scala.util.Properties.{ envOrNone => env } lazy val hadoopVersion = env("SHARK_HADOOP_VERSION") orElse env("SPARK_HADOOP_VERSION") getOrElse DEFAULT_HADOOP_VERSION - // Whether to build Shark with Yarn support - val YARN_ENABLED = scala.util.Properties.envOrNone("SHARK_YARN") match { - case None => false - case Some(v) => v.toBoolean - } + val YARN_ENABLED = env("SHARK_YARN").getOrElse("false").toBoolean // Whether to build Shark with Tachyon jar. val TACHYON_ENABLED = false From 876de55a5673d319aa8b9628a250aec690e39954 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 18 Sep 2013 11:04:15 +0800 Subject: [PATCH 038/331] Fix hadoop conf dir path for Hadoop 2.x --- run | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/run b/run index 04084830..48843652 100755 --- a/run +++ b/run @@ -106,9 +106,10 @@ SPARK_CLASSPATH+=":$SHARK_HOME/target/scala-$SCALA_VERSION/test-classes" if [ "x$HADOOP_HOME" == "x" ] ; then - echo "No HADOOP_HOME specified. Shark will run in local-mode" + echo "No HADOOP_HOME specified. Shark will run in local-mode" else - SPARK_CLASSPATH+=:$HADOOP_HOME/conf + SPARK_CLASSPATH+=:$HADOOP_HOME/etc/hadoop + SPARK_CLASSPATH+=:$HADOOP_HOME/conf fi From c880abe969ecc8371a176bafd77ef78a697db28c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 18 Sep 2013 14:53:44 -0700 Subject: [PATCH 039/331] Took log4j init out of SharkContext. --- src/main/scala/shark/SharkContext.scala | 8 -------- src/main/scala/shark/repl/Main.scala | 10 ++++++++++ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 3f896211..0f670845 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -24,8 +24,6 @@ import scala.collection.Map import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessor import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory @@ -179,12 +177,6 @@ object SharkContext { @transient val hiveconf = new HiveConf(classOf[SessionState]) Utils.setAwsCredentials(hiveconf) - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } - @transient val sessionState = new SessionState(hiveconf) sessionState.out = new PrintStream(System.out, true, "UTF-8") sessionState.err = new PrintStream(System.out, true, "UTF-8") diff --git a/src/main/scala/shark/repl/Main.scala b/src/main/scala/shark/repl/Main.scala index 1fa22da5..890a74ef 100755 --- a/src/main/scala/shark/repl/Main.scala +++ b/src/main/scala/shark/repl/Main.scala @@ -17,11 +17,21 @@ package shark.repl +import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException + + /** * Shark's REPL entry point. */ object Main { + try { + LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => // Ignore the error. + } + private var _interp: SharkILoop = null def interp = _interp From f7571073d4909e9a54a8420a48178b394693be9e Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Wed, 18 Sep 2013 21:27:29 -0700 Subject: [PATCH 040/331] scalastyle checks for sbt. All warnings for now. Execute using 'sbt scalastyle' --- project/SharkBuild.scala | 2 +- project/plugins.sbt | 4 ++ scalastyle-config.xml | 118 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 123 insertions(+), 1 deletion(-) create mode 100644 scalastyle-config.xml diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 088e8fc0..a96b2d52 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -128,7 +128,7 @@ object SharkBuild extends Build { "com.novocode" % "junit-interface" % "0.8" % "test") ++ (if (YARN_ENABLED) Some("org.apache.spark" %% "spark-yarn" % SPARK_VERSION) else None).toSeq ++ (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop) ) else None).toSeq - ) + ) ++ org.scalastyle.sbt.ScalastylePlugin.Settings def assemblyProjSettings = Seq( jarName in assembly <<= version map { v => "shark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" } diff --git a/project/plugins.sbt b/project/plugins.sbt index d06b220d..5b2a7785 100755 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -15,6 +15,8 @@ addSbtPlugin("org.ensime" % "ensime-sbt-cmd" % "0.1.1") +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.3.2") + addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.4.0") @@ -24,3 +26,5 @@ addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") resolvers += Resolver.url( "sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) + +resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/" diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 00000000..a65482e0 --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,118 @@ + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From 359969ecf1a3e1c75f77990454ec1e2926a7f3e6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 19 Sep 2013 14:45:36 -0700 Subject: [PATCH 041/331] New spark naming --- src/main/scala/shark/api/RDDTable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index a03622c2..8a3ce1bc 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import shark.SharkEnv import shark.memstore2.{TablePartitionStats, TablePartition, TablePartitionBuilder} import shark.util.HiveUtils -import spark.RDD +import org.apache.spark.rdd.RDD class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { From bd362f1bdbf9529388b8f9d1b1177a12d8aefc1c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 15:55:34 -0700 Subject: [PATCH 042/331] Revise some added comments. --- .../memstore2/MemoryMetadataManager.scala | 6 +-- .../scala/shark/memstore2/MemoryTable.scala | 2 +- .../shark/parse/SharkSemanticAnalyzer.scala | 42 +++++++++---------- 3 files changed, 23 insertions(+), 27 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index ce3f52a8..53835e89 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -54,8 +54,8 @@ class MemoryMetadataManager { def put(key: String, rdd: RDD[_]) { if (!_keyToMemoryTable.contains(key.toLowerCase)) { // TODO(harvey): Remove this once CREATE TABLE/CTAS handling involves calling add(). For now, - // CTAS is done in the MemoryStoreSinkOperator, which also "adds" the RDD for - // the created table to MemoryMetadataManager. + // CTAS result caching is done by MemoryStoreSinkOperator, which calls this + // put() method. add(key, false /* isHivePartitioned */) } _keyToMemoryTable(key.toLowerCase).tableRDD = rdd @@ -101,7 +101,7 @@ class MemoryMetadataManager { * @param key Name of the table to drop. * @return Option::isEmpty() is true of there is no MemoryTable corresponding to 'key' in * _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will be - * a UnionRDD comprising all RDDs for all Hive-partitions. + * a UnionRDD comprising RDDs that represent the Hive-partitions. */ def unpersist(key: String): Option[RDD[_]] = { def unpersistRDD(rdd: RDD[_]) { diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 6a8a8e13..60e1360a 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -70,4 +70,4 @@ private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: B "Table " + tableName + " is not Hive-partitioned. Use tableRDD() to get its RDD.") _hivePartitionRDDs } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index ec94d3f9..2f05bb1d 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -90,11 +90,13 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // Do post-Hive analysis of the CREATE TABLE (e.g detect caching mode). analyzeCreateTable(ast, qb) match { case Some(selectStmtASTNode) => { - // The 'child' is set to the SELECT statement root node, with is a HiveParer.HIVE_QUERY. + // Set the 'child' to reference the SELECT statement root node, with is a + // HiveParer.HIVE_QUERY. child = selectStmtASTNode - // Hive will generate MapReduce tasks for the SELECT. Avoid executing those tasks by - // reset()-ing some Hive SemanticAnalyzer state after phase 1 of analysis below. - // TODO(harvey): This might be too much. SharkSemanticAnalyzer could directly clear + // Hive's super.analyzeInternal() generates MapReduce tasks for the SELECT. Avoid + // executing those tasks by reset()-ing some Hive SemanticAnalyzer state after phase 1 of + // analysis below. + // TODO(harvey): This might be too much. SharkSemanticAnalyzer could just clear // 'rootTasks', since it's a protected field. shouldReset = true } @@ -108,12 +110,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with SessionState.get().setCommandType(HiveOperation.QUERY) } - // Invariant: At this point, the command includes a query - it's ASTNode has a - // HiveParser.TOK_QUERY somewhere). + // Invariant: At this point, the command will execute a query (i.e., its AST contains a + // HiveParser.TOK_QUERY node). // Continue analyzing from the child ASTNode. - // TODO(harvey): Look into whether doPhase1() can be skipped for CTAS. The - // 'super.analyzeInternal()' call above already calls it. if (!doPhase1(child, qb, initPhase1Ctx())) { return } @@ -156,9 +156,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val terminalOpSeq = { if (qb.getParseInfo.isInsertToTable && !qb.isCTAS) { // Handle an INSERT into a cached table. - // TODO(harvey): Detect the RDD's cache mode. This could be easily be done by using the - // cachedTableName below. The cache mode property could be added to - // MemoryTable. hiveSinkOps.map { hiveSinkOp => val tableName = hiveSinkOp.asInstanceOf[HiveFileSinkOperator].getConf().getTableInfo() .getTableName() @@ -341,12 +338,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // SELECT statement var selectStmtASTNode: Option[ASTNode] = None - // TODO(harvey): Probably don't need this. We might be able to reuse the QB passed into this - // method, as long as it was created after the super.analyzeInternal() call. - // That QB and the createTableDesc should have everything (e.g. isCTAS(), - // partCols). Note that the QB might not be accessible from getParseContext(), - // since the SemanticAnalyzer#analyzeInternal() doesn't set (this.qb = qb) for a - // non-CTAS. + // TODO(harvey): We might be able to reuse the QB passed into this method, as long as it was + // created after the super.analyzeInternal() call. That QB and the createTableDesc + // should have everything (e.g. isCTAS(), partCols). Note that the QB might not be + // accessible from getParseContext(), since the SemanticAnalyzer#analyzeInternal() + // doesn't set (this.qb = qb) for a non-CTAS. var isCTAS = false var isHivePartitioned = false @@ -364,10 +360,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } // The 'createTableDesc' can be NULL if the command is a ... - // 1) syntactically valid CREATE TABLE statement. The table specified may or may not already - // exist. If the table already exists, then an exception is thrown by the DDLTask that's - // executed after semantic analysis. - // 2) valid CTAS statement with an IF NOT EXISTS condition, and the specified table already + // 1) syntactically valid CREATE TABLE statement. Note that the table specified may or may not + // already exist. If the table already exists, then an exception is thrown by the DDLTask + // that's executed after semantic analysis. + // 2) valid CTAS statement with an IF NOT EXISTS condition and the specified table already // exists. If the table to-be-created already exists, and the CTAS statement does not // have an IF NOT EXISTS condition, then an exception will be thrown by // SemanticAnalzyer#analyzeInternal(). @@ -375,8 +371,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with if (isCTAS && createTableDesc != null) { val tableName = createTableDesc.getTableName val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) - // Note: the CreateTableDesc's table properties are Java Maps, but the TableDesc's table - // properties, which are used during execution, are Java Properties. + // The CreateTableDesc's table properties are Java Maps, but the TableDesc's table properties, + // which are used during execution, are Java Properties. val createTableProperties: JavaMap[String, String] = createTableDesc.getTblProps() // There are two cases that will enable caching: From 6b4f0cf2881449c3a7413cd42e84a67e66d08641 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 18:46:53 -0700 Subject: [PATCH 043/331] Minor syntax changes to RDDTable. --- src/main/scala/shark/api/RDDTable.scala | 11 +++++------ .../scala/shark/execution/TableScanOperator.scala | 8 ++++---- src/main/scala/shark/util/HiveUtils.scala | 9 +++++---- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 8a3ce1bc..a23e1632 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -53,24 +53,23 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() + val isDDLStatementSuccessful = HiveUtils.createTable(tableName, fields, manifests) + // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. - if (HiveUtils.createTable(tableName, fields, manifests)) { + if (isDDLStatementSuccessful) { // Force evaluate to put the data in memory. SharkEnv.memoryMetadataManager.put(tableName, rdd) rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) // Gather the partition statistics. SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) - - true - } else { - false } + return isDDLStatementSuccessful } } -object RDDTable { +object RDDToTable { private type M[T] = ClassManifest[T] private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 9115bac7..ffed99f3 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -127,7 +127,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO throw(new QueryExecutionException("Cached table not found")) } logInfo("Loading table " + tableKey + " from Spark block manager") - createPrunedRdd(tableKey, rdd) + return createPrunedRdd(tableKey, rdd) } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { @@ -145,10 +145,10 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO logInfo("Loading table " + tableKey + " stats from Tachyon.") SharkEnv.memoryMetadataManager.putStats(tableKey, indexToStats) } - createPrunedRdd(tableKey, SharkEnv.tachyonUtil.createRDD(tableKey)) + return createPrunedRdd(tableKey, SharkEnv.tachyonUtil.createRDD(tableKey)) } else { // Table is a Hive table on HDFS (or other Hadoop storage). - super.execute() + return super.execute() } } @@ -199,7 +199,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO rdd } - prunedRdd.mapPartitions { iter => + return prunedRdd.mapPartitions { iter => if (iter.hasNext) { val tablePartition = iter.next.asInstanceOf[TablePartition] tablePartition.prunedIterator(columnsUsed) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index d7542f90..6b6254c6 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -55,8 +55,7 @@ private[shark] object HiveUtils { def createTable( tableName: String, columnNames: Seq[String], - columnTypes: Seq[ClassManifest[_]]): Boolean = - { + columnTypes: Seq[ClassManifest[_]]): Boolean = { val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") } @@ -78,6 +77,8 @@ private[shark] object HiveUtils { task.setWork(work) // Hive returns 0 if the create table command is executed successfully. - task.execute(null) == 0 + val taskExecutionStatus: Int = task.execute(null) + + return taskExecutionStatus } -} \ No newline at end of file +} From e578307ef895e56e7a694605c010006eb31f8648 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 Sep 2013 20:28:09 -0700 Subject: [PATCH 044/331] Recognize Java primitive types in RDDTable. --- src/main/scala/shark/util/HiveUtils.scala | 28 +++++++++++------------ 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index d7542f90..232d55b3 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -26,27 +26,27 @@ import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} import org.apache.hadoop.hive.ql.plan.{DDLWork, CreateTableDesc} import org.apache.hadoop.hive.metastore.api.FieldSchema -import shark.api.DataTypes +import shark.api.{DataType, DataTypes} import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.conf.HiveConf private[shark] object HiveUtils { - private val timestampManfiest = classManifest[java.sql.Timestamp] - private val stringManifest = classManifest[String] + def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = { + getJavaPrimitiveObjectInspector(DataTypes.fromManifest(m)) + } - def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = m match { - case Manifest.Boolean => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector - case Manifest.Byte => PrimitiveObjectInspectorFactory.javaByteObjectInspector - case Manifest.Short => PrimitiveObjectInspectorFactory.javaShortObjectInspector - case Manifest.Int => PrimitiveObjectInspectorFactory.javaIntObjectInspector - case Manifest.Long => PrimitiveObjectInspectorFactory.javaLongObjectInspector - case Manifest.Float => PrimitiveObjectInspectorFactory.javaFloatObjectInspector - case Manifest.Double => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector - case Manifest.Unit => PrimitiveObjectInspectorFactory.javaVoidObjectInspector - case `timestampManfiest` => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector - case `stringManifest` => PrimitiveObjectInspectorFactory.javaStringObjectInspector + def getJavaPrimitiveObjectInspector(t: DataType): PrimitiveObjectInspector = t match { + case DataTypes.BOOLEAN => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case DataTypes.TINYINT => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case DataTypes.SMALLINT => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case DataTypes.INT => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case DataTypes.BIGINT => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case DataTypes.FLOAT => PrimitiveObjectInspectorFactory.javaFloatObjectInspector + case DataTypes.DOUBLE => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case DataTypes.TIMESTAMP => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DataTypes.STRING => PrimitiveObjectInspectorFactory.javaStringObjectInspector } /** From fdb4b9c31bd5dd91f14b0be9a042d28b7feb3a8e Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 20:49:19 -0700 Subject: [PATCH 045/331] Remove table from Shark and Hive metadata if the Spark job for RDD => Shark table RDD transformation fails. --- src/main/scala/shark/api/RDDTable.scala | 23 +++++++--- src/main/scala/shark/util/HiveUtils.scala | 54 ++++++++++++++++------- 2 files changed, 55 insertions(+), 22 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index a23e1632..0db90722 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -51,25 +51,36 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) Iterator(builder.build()) - }.persist() + }.persist(self.getStorageLevel) - val isDDLStatementSuccessful = HiveUtils.createTable(tableName, fields, manifests) + var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. - if (isDDLStatementSuccessful) { + if (isSucessfulCreateTable) { // Force evaluate to put the data in memory. SharkEnv.memoryMetadataManager.put(tableName, rdd) - rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + try { + rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + } catch { + case _ => { + // Intercept the exception thrown by SparkContext#runJob() and return silently. The + // exception message should be printed to the console by DDLTask#execute(). + HiveUtils.dropTableInHive(tableName) + // Drop the table entry from MemoryMetadataManager. + SharkEnv.unpersist(tableName) + isSucessfulCreateTable = false + } + } // Gather the partition statistics. SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) } - return isDDLStatementSuccessful + return isSucessfulCreateTable } } -object RDDToTable { +object RDDTable { private type M[T] = ClassManifest[T] private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 6b6254c6..81bd2f7d 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} -import org.apache.hadoop.hive.ql.plan.{DDLWork, CreateTableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLDesc, DDLWork, DropTableDesc} import org.apache.hadoop.hive.metastore.api.FieldSchema import shark.api.DataTypes @@ -52,7 +52,7 @@ private[shark] object HiveUtils { /** * Execute the create table DDL operation against Hive's metastore. */ - def createTable( + def createTableInHive( tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]]): Boolean = { @@ -61,24 +61,46 @@ private[shark] object HiveUtils { } // Setup the create table descriptor with necessary information. - val desc = new CreateTableDesc() - desc.setTableName(tableName) - desc.setCols(new JavaArrayList[FieldSchema](schema)) - desc.setTblProps(Map("shark.cache" -> "heap")) - desc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") - desc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") - desc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) - desc.setNumBuckets(-1) + val createTbleDesc = new CreateTableDesc() + createTbleDesc.setTableName(tableName) + createTbleDesc.setCols(new JavaArrayList[FieldSchema](schema)) + createTbleDesc.setTblProps(Map("shark.cache" -> "heap")) + createTbleDesc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") + createTbleDesc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") + createTbleDesc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) + createTbleDesc.setNumBuckets(-1) // Execute the create table against the metastore. - val work = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], desc) - val task = new DDLTask + val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], + new JavaHashSet[WriteEntity], + createTbleDesc) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + return (taskExecutionStatus == 0) + } + + def dropTableInHive( + tableName: String): Boolean = { + // Setup the drop table descriptor with necessary information. + val dropTblDesc = new DropTableDesc( + tableName, + false /* expectView. Should probably be named "isView".*/, + false /* ifExists */, + false /* stringPartitionColumns */) + + // Execute the drop table against the metastore. + val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], + new JavaHashSet[WriteEntity], + dropTblDesc) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + return (taskExecutionStatus == 0) + } + + def executeDDLTaskDirectly(ddlWork: DDLWork): Int = { + val task = new DDLTask() task.initialize(new HiveConf, null, null) - task.setWork(work) + task.setWork(ddlWork) // Hive returns 0 if the create table command is executed successfully. - val taskExecutionStatus: Int = task.execute(null) - - return taskExecutionStatus + return task.execute(null) } } From 30dd3b3b628e9f24827a0826c3992ba5c91886b2 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 20:53:09 -0700 Subject: [PATCH 046/331] Merge remote-tracking branch 'ampshark/rddtable' into rddtable --- src/main/scala/shark/SharkContext.scala | 8 ------ src/main/scala/shark/api/RDDTable.scala | 4 +-- src/main/scala/shark/repl/Main.scala | 10 ++++++++ src/main/scala/shark/util/HiveUtils.scala | 30 +++++++++++------------ 4 files changed, 27 insertions(+), 25 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 3f896211..0f670845 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -24,8 +24,6 @@ import scala.collection.Map import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessor import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory @@ -179,12 +177,6 @@ object SharkContext { @transient val hiveconf = new HiveConf(classOf[SessionState]) Utils.setAwsCredentials(hiveconf) - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } - @transient val sessionState = new SessionState(hiveconf) sessionState.out = new PrintStream(System.out, true, "UTF-8") sessionState.err = new PrintStream(System.out, true, "UTF-8") diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 0db90722..820ea65d 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -63,8 +63,8 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } catch { case _ => { - // Intercept the exception thrown by SparkContext#runJob() and return silently. The - // exception message should be printed to the console by DDLTask#execute(). + // Intercept the exception thrown by SparkContext#runJob() and handle it silently. The + // exception message should already be printed to the console by DDLTask#execute(). HiveUtils.dropTableInHive(tableName) // Drop the table entry from MemoryMetadataManager. SharkEnv.unpersist(tableName) diff --git a/src/main/scala/shark/repl/Main.scala b/src/main/scala/shark/repl/Main.scala index 1fa22da5..890a74ef 100755 --- a/src/main/scala/shark/repl/Main.scala +++ b/src/main/scala/shark/repl/Main.scala @@ -17,11 +17,21 @@ package shark.repl +import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException + + /** * Shark's REPL entry point. */ object Main { + try { + LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => // Ignore the error. + } + private var _interp: SharkILoop = null def interp = _interp diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 81bd2f7d..8f2f1473 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -26,27 +26,27 @@ import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLDesc, DDLWork, DropTableDesc} import org.apache.hadoop.hive.metastore.api.FieldSchema -import shark.api.DataTypes +import shark.api.{DataType, DataTypes} import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.conf.HiveConf private[shark] object HiveUtils { - private val timestampManfiest = classManifest[java.sql.Timestamp] - private val stringManifest = classManifest[String] + def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = { + getJavaPrimitiveObjectInspector(DataTypes.fromManifest(m)) + } - def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = m match { - case Manifest.Boolean => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector - case Manifest.Byte => PrimitiveObjectInspectorFactory.javaByteObjectInspector - case Manifest.Short => PrimitiveObjectInspectorFactory.javaShortObjectInspector - case Manifest.Int => PrimitiveObjectInspectorFactory.javaIntObjectInspector - case Manifest.Long => PrimitiveObjectInspectorFactory.javaLongObjectInspector - case Manifest.Float => PrimitiveObjectInspectorFactory.javaFloatObjectInspector - case Manifest.Double => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector - case Manifest.Unit => PrimitiveObjectInspectorFactory.javaVoidObjectInspector - case `timestampManfiest` => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector - case `stringManifest` => PrimitiveObjectInspectorFactory.javaStringObjectInspector + def getJavaPrimitiveObjectInspector(t: DataType): PrimitiveObjectInspector = t match { + case DataTypes.BOOLEAN => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case DataTypes.TINYINT => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case DataTypes.SMALLINT => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case DataTypes.INT => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case DataTypes.BIGINT => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case DataTypes.FLOAT => PrimitiveObjectInspectorFactory.javaFloatObjectInspector + case DataTypes.DOUBLE => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case DataTypes.TIMESTAMP => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DataTypes.STRING => PrimitiveObjectInspectorFactory.javaStringObjectInspector } /** @@ -83,7 +83,7 @@ private[shark] object HiveUtils { // Setup the drop table descriptor with necessary information. val dropTblDesc = new DropTableDesc( tableName, - false /* expectView. Should probably be named "isView".*/, + false /* expectView */, false /* ifExists */, false /* stringPartitionColumns */) From 955d5e5f9705db395d0ff5f4b685beca8598f1ba Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 21:07:58 -0700 Subject: [PATCH 047/331] Don't persist new Shark RDD with same storage level as parent. --- src/main/scala/shark/api/RDDTable.scala | 2 +- src/main/scala/shark/util/HiveUtils.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 820ea65d..31070873 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -51,7 +51,7 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) Iterator(builder.build()) - }.persist(self.getStorageLevel) + }.persist() var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 8f2f1473..5a9e686e 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} -import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLDesc, DDLWork, DropTableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} import org.apache.hadoop.hive.metastore.api.FieldSchema import shark.api.{DataType, DataTypes} From c281e82052abc87c7c92d593819e432d7d00d120 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 Sep 2013 22:23:15 -0700 Subject: [PATCH 048/331] A smarter execution strategy for limit. --- .../shark/execution/FileSinkOperator.scala | 37 +++++++++++++------ 1 file changed, 25 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index cfd93640..c5887136 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -143,24 +143,36 @@ class FileSinkOperator extends TerminalOperator with Serializable { parentOperators.head match { case op: LimitOperator => - // If there is a limit operator, let's only run one partition at a time to avoid - // launching too many tasks. + // If there is a limit operator, let's run two partitions first. Once we finished running + // the first two partitions, we use that to estimate how many more partitions we need to + // run to satisfy the limit. + val limit = op.limit - val numPartitions = rdd.partitions.length - var totalRows = 0 - var nextPartition = 0 - while (totalRows < limit && nextPartition < numPartitions) { - // Run one partition and get back the number of rows processed there. - totalRows += rdd.context.runJob( + val totalParts = rdd.partitions.length + var rowsFetched = 0L + var partsFetched = 0 + while (rowsFetched < limit && partsFetched < totalParts) { + var numPartsToTry = 2 + if (partsFetched > 0) { + if (rowsFetched == 0) { + numPartsToTry = totalParts - 2 + } else { + numPartsToTry = (limit * partsFetched / rowsFetched).toInt + } + } + numPartsToTry = math.max(0, totalParts - 2) // guard against negative num of partitions + + rowsFetched += rdd.context.runJob( rdd, FileSinkOperator.executeProcessFileSinkPartition(this), - Seq(nextPartition), + partsFetched until math.min(partsFetched + numPartsToTry, totalParts), allowLocal = false).sum - nextPartition += 1 + partsFetched += numPartsToTry } case _ => - val rows = rdd.context.runJob(rdd, FileSinkOperator.executeProcessFileSinkPartition(this)) + val rows: Array[Long] = rdd.context.runJob( + rdd, FileSinkOperator.executeProcessFileSinkPartition(this)) logInfo("Total number of rows written: " + rows.sum) } @@ -171,9 +183,10 @@ class FileSinkOperator extends TerminalOperator with Serializable { object FileSinkOperator { + // Write each partition's output to HDFS, and return the number of rows written. def executeProcessFileSinkPartition(operator: FileSinkOperator) = { val op = OperatorSerializationWrapper(operator) - def writeFiles(context: TaskContext, iter: Iterator[_]): Int = { + def writeFiles(context: TaskContext, iter: Iterator[_]): Long = { op.logDebug("Started executing mapPartitions for operator: " + op) op.logDebug("Input object inspectors: " + op.objectInspectors) From ad6bca1694fdb24bb0981aa57ac43b131ff445a0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 Sep 2013 22:36:05 -0700 Subject: [PATCH 049/331] Fixed a bug in numPartsToTry calculation. --- src/main/scala/shark/execution/FileSinkOperator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index c5887136..92082318 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -160,7 +160,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { numPartsToTry = (limit * partsFetched / rowsFetched).toInt } } - numPartsToTry = math.max(0, totalParts - 2) // guard against negative num of partitions + numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions rowsFetched += rdd.context.runJob( rdd, From 38c5c247a55cf04309564cd2f7cc71de1abf3c87 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 Sep 2013 22:46:33 -0700 Subject: [PATCH 050/331] Overestimate by 50%. --- src/main/scala/shark/execution/FileSinkOperator.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index 92082318..fa811340 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -152,12 +152,17 @@ class FileSinkOperator extends TerminalOperator with Serializable { var rowsFetched = 0L var partsFetched = 0 while (rowsFetched < limit && partsFetched < totalParts) { + // The number of partitions to try in this iteration. It is ok for this number to be + // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 2 if (partsFetched > 0) { + // If we didn't find any rows after the first iteration, just try all partitions next. + // Otherwise, interpolate the number of partitions we need to try, but overestimate it + // by 50%. if (rowsFetched == 0) { numPartsToTry = totalParts - 2 } else { - numPartsToTry = (limit * partsFetched / rowsFetched).toInt + numPartsToTry = (1.5 * limit * partsFetched / rowsFetched).toInt } } numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions From 784845280d9eadba993767c66d70edeeb90e7408 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Sun, 22 Sep 2013 15:07:30 +0800 Subject: [PATCH 051/331] Do not init spark context when in remote mode --- src/main/scala/shark/SharkCliDriver.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index ee316b66..2dd66073 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -222,7 +222,7 @@ object SharkCliDriver { class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper { - private val ss = SessionState.get() + private val ss = SessionState.get().asInstanceOf[CliSessionState] private val LOG = LogFactory.getLog("CliDriver") @@ -234,7 +234,9 @@ class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper // Force initializing SharkEnv. This is put here but not object SharkCliDriver // because the Hive unit tests do not go through the main() code path. - SharkEnv.init() + if (!ss.isRemoteMode()) { + SharkEnv.init() + } if(loadRdds) CachedTableRecovery.loadAsRdds(processCmd(_)) From c779029bb515741102fc3434bbb37584f5b7df6e Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 20 Sep 2013 17:13:32 -0700 Subject: [PATCH 052/331] SHARK-193: reducing Shark's logging verbosity --- src/main/scala/shark/SharkDriver.scala | 4 ++-- src/main/scala/shark/SharkEnv.scala | 4 ++-- src/main/scala/shark/execution/CoGroupedRDD.scala | 4 ++-- src/main/scala/shark/execution/FileSinkOperator.scala | 2 +- .../shark/execution/GroupByPostShuffleOperator.scala | 4 ++-- .../scala/shark/execution/GroupByPreShuffleOperator.scala | 2 +- src/main/scala/shark/execution/HiveTopOperator.scala | 6 +++--- src/main/scala/shark/execution/MapJoinOperator.scala | 8 ++++---- src/main/scala/shark/execution/ScriptOperator.scala | 2 +- src/main/scala/shark/execution/SharkExplainTask.scala | 2 +- src/main/scala/shark/execution/SparkTask.scala | 2 +- src/main/scala/shark/execution/TableScanOperator.scala | 4 ++-- src/main/scala/shark/execution/UnionOperator.scala | 2 +- src/main/scala/shark/parse/SharkSemanticAnalyzer.scala | 8 ++++---- 14 files changed, 27 insertions(+), 27 deletions(-) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index d6c9687a..6a11aa2b 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -51,7 +51,7 @@ private[shark] object SharkDriver extends LogHelper { // A dummy static method so we can make sure the following static code are executed. def runStaticCode() { - logInfo("Initializing object SharkDriver") + logDebug("Initializing object SharkDriver") } def registerSerDe(serdeClass: Class[_ <: SerDe]) { @@ -198,7 +198,7 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe sem.analyze(tree, context) } - logInfo("Semantic Analysis Completed") + logDebug("Semantic Analysis Completed") sem.validate() diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 55d646e0..3e5f9d00 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -83,7 +83,7 @@ object SharkEnv extends LogHelper { new JavaSharkContext(initWithSharkContext(newSc.sharkCtx)) } - logInfo("Initializing SharkEnv") + logDebug("Initializing SharkEnv") System.setProperty("spark.serializer", classOf[SparkKryoSerializer].getName) System.setProperty("spark.kryo.registrator", classOf[KryoRegistrator].getName) @@ -128,7 +128,7 @@ object SharkEnv extends LogHelper { /** Cleans up and shuts down the Shark environments. */ def stop() { - logInfo("Shutting down Shark Environment") + logDebug("Shutting down Shark Environment") // Stop the SparkContext if (SharkEnv.sc != null) { sc.stop() diff --git a/src/main/scala/shark/execution/CoGroupedRDD.scala b/src/main/scala/shark/execution/CoGroupedRDD.scala index e803db90..4b5da641 100644 --- a/src/main/scala/shark/execution/CoGroupedRDD.scala +++ b/src/main/scala/shark/execution/CoGroupedRDD.scala @@ -72,10 +72,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) override def getDependencies: Seq[Dependency[_]] = { rdds.map { rdd => if (rdd.partitioner == Some(part)) { - logInfo("Adding one-to-one dependency with " + rdd) + logDebug("Adding one-to-one dependency with " + rdd) new OneToOneDependency(rdd) } else { - logInfo("Adding shuffle dependency with " + rdd) + logDebug("Adding shuffle dependency with " + rdd) new ShuffleDependency[Any, Any](rdd, part, SharkEnv.shuffleSerializerName) } } diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index fa811340..641c827a 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -178,7 +178,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { case _ => val rows: Array[Long] = rdd.context.runJob( rdd, FileSinkOperator.executeProcessFileSinkPartition(this)) - logInfo("Total number of rows written: " + rows.sum) + logDebug("Total number of rows written: " + rows.sum) } hiveOp.jobClose(localHconf, true, new JobCloseFeedBack) diff --git a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala index 737f5b95..c180fa34 100755 --- a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala @@ -231,7 +231,7 @@ class GroupByPostShuffleOperator extends GroupByPreShuffleOperator with HiveTopO } def sortAggregate(iter: Iterator[_]) = { - logInfo("Running Post Shuffle Group-By") + logDebug("Running Post Shuffle Group-By") if (iter.hasNext) { // Sort based aggregation iterator. @@ -401,7 +401,7 @@ class GroupByPostShuffleOperator extends GroupByPreShuffleOperator with HiveTopO def hashAggregate(iter: Iterator[_]) = { // TODO: use MutableBytesWritable to avoid the array copy. val bytes = new BytesWritable() - logInfo("Running Post Shuffle Group-By") + logDebug("Running Post Shuffle Group-By") val outputCache = new Array[Object](keyFields.length + aggregationEvals.length) // The reusedRow is used to conform to Hive's expected row format. diff --git a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala index f19ec7a6..055f682a 100755 --- a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala @@ -107,7 +107,7 @@ class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { } override def processPartition(split: Int, iter: Iterator[_]) = { - logInfo("Running Pre-Shuffle Group-By") + logDebug("Running Pre-Shuffle Group-By") var numRowsInput = 0 var numRowsHashTbl = 0 var useHashAggr = true diff --git a/src/main/scala/shark/execution/HiveTopOperator.scala b/src/main/scala/shark/execution/HiveTopOperator.scala index 825c9f90..d9f611f4 100755 --- a/src/main/scala/shark/execution/HiveTopOperator.scala +++ b/src/main/scala/shark/execution/HiveTopOperator.scala @@ -51,14 +51,14 @@ trait HiveTopOperator extends LogHelper { * Initialize the Hive operator when all input object inspectors are ready. */ def initializeHiveTopOperator() { - logInfo("Started executing " + self + " initializeHiveTopOperator()") + logDebug("Started executing " + self + " initializeHiveTopOperator()") // Call initializeDownStreamHiveOperators() of upstream operators that are // ReduceSink so we can get the proper input object inspectors and serdes. val reduceSinkParents = self.parentOperators.filter(_.isInstanceOf[ReduceSinkOperator]) reduceSinkParents.foreach { parent => parent.asInstanceOf[ReduceSinkOperator].initializeDownStreamHiveOperator() - logInfo("parent : " + parent) + logDebug("parent : " + parent) } // Only do initialize if all our input inspectors are ready. We use > @@ -86,7 +86,7 @@ trait HiveTopOperator extends LogHelper { self.hiveOp.initialize(hconf, objectInspectorArray) } - logInfo("Finished executing " + self + " initializeHiveTopOperator()") + logDebug("Finished executing " + self + " initializeHiveTopOperator()") } def setInputObjectInspector(tag: Int, objectInspector: ObjectInspector) { diff --git a/src/main/scala/shark/execution/MapJoinOperator.scala b/src/main/scala/shark/execution/MapJoinOperator.scala index 32cdbcfe..400415b5 100755 --- a/src/main/scala/shark/execution/MapJoinOperator.scala +++ b/src/main/scala/shark/execution/MapJoinOperator.scala @@ -92,8 +92,8 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato } override def combineMultipleRdds(rdds: Seq[(Int, RDD[_])]): RDD[_] = { - logInfo("%d small tables to map join a large table (%d)".format(rdds.size - 1, posBigTable)) - logInfo("Big table alias " + bigTableAlias) + logDebug("%d small tables to map join a large table (%d)".format(rdds.size - 1, posBigTable)) + logDebug("Big table alias " + bigTableAlias) val op1 = OperatorSerializationWrapper(this) @@ -102,7 +102,7 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato // Build hash tables for the small tables. val hashtables = rdds.zipWithIndex.filter(_._2 != bigTableAlias).map { case ((_, rdd), pos) => - logInfo("Creating hash table for input %d".format(pos)) + logDebug("Creating hash table for input %d".format(pos)) // First compute the keys and values of the small RDDs on slaves. // We need to do this before collecting the RDD because the RDD might @@ -141,7 +141,7 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato if(storageLevel == StorageLevel.NONE) rddForHash.unpersist() - logInfo("wrappedRows size:" + wrappedRows.size) + logDebug("wrappedRows size:" + wrappedRows.size) val collectTime = System.currentTimeMillis() - startCollect logInfo("HashTable collect took " + collectTime + " ms") diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index f8b56c82..73c04f08 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -64,7 +64,7 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { val (command, envs) = getCommandAndEnvs() val outRecordReaderClass: Class[_ <: RecordReader] = hiveOp.getConf().getOutRecordReaderClass() val inRecordWriterClass: Class[_ <: RecordWriter] = hiveOp.getConf().getInRecordWriterClass() - logInfo("Using %s and %s".format(outRecordReaderClass, inRecordWriterClass)) + logDebug("Using %s and %s".format(outRecordReaderClass, inRecordWriterClass)) // Deserialize the output from script back to what Hive understands. inputRdd.mapPartitions { part => diff --git a/src/main/scala/shark/execution/SharkExplainTask.scala b/src/main/scala/shark/execution/SharkExplainTask.scala index 2462a67b..5d69e3f6 100755 --- a/src/main/scala/shark/execution/SharkExplainTask.scala +++ b/src/main/scala/shark/execution/SharkExplainTask.scala @@ -50,7 +50,7 @@ class SharkExplainTask extends Task[SharkExplainWork] with java.io.Serializable val hiveExplainTask = new ExplainTask override def execute(driverContext: DriverContext): Int = { - logInfo("Executing " + this.getClass.getName()) + logDebug("Executing " + this.getClass.getName()) hiveExplainTask.setWork(work) try { diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index 32241a47..e36b4c20 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -54,7 +54,7 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { def tableRdd: Option[TableRDD] = _tableRdd override def execute(driverContext: DriverContext): Int = { - logInfo("Executing " + this.getClass.getName) + logDebug("Executing " + this.getClass.getName) val ctx = driverContext.getCtx() diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 9115bac7..27247503 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -215,13 +215,13 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO */ override def preprocessRdd(rdd: RDD[_]): RDD[_] = { if (table.isPartitioned) { - logInfo("Making %d Hive partitions".format(parts.size)) + logDebug("Making %d Hive partitions".format(parts.size)) makePartitionRDD(rdd) } else { val tablePath = table.getPath.toString val ifc = table.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - logInfo("Table input: %s".format(tablePath)) + logDebug("Table input: %s".format(tablePath)) createHadoopRdd(tablePath, ifc) } } diff --git a/src/main/scala/shark/execution/UnionOperator.scala b/src/main/scala/shark/execution/UnionOperator.scala index 2e46a004..49157ef0 100755 --- a/src/main/scala/shark/execution/UnionOperator.scala +++ b/src/main/scala/shark/execution/UnionOperator.scala @@ -89,7 +89,7 @@ class UnionOperator extends NaryOperator[HiveUnionOperator] { // We reuse needsTransform from Hive because the comparison of object // inspectors are hard once we send object inspectors over the wire. needsTransform.zipWithIndex.filter(_._1).foreach { case(transform, p) => - logInfo("Union Operator needs to transform row from parent[%d] from %s to %s".format( + logDebug("Union Operator needs to transform row from parent[%d] from %s to %s".format( p, objectInspectors(p), outputObjInspector)) } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 2dfbc695..aa6ea812 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -73,7 +73,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with init(pctx) var child: ASTNode = ast - logInfo("Starting Shark Semantic Analysis") + logDebug("Starting Shark Semantic Analysis") //TODO: can probably reuse Hive code for this // analyze create table command @@ -146,9 +146,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // Used to protect against recursive views in getMetaData(). SharkSemanticAnalyzer.viewsExpandedField.set(this, new ArrayList[String]()) - logInfo("Completed phase 1 of Shark Semantic Analysis") + logDebug("Completed phase 1 of Shark Semantic Analysis") getMetaData(qb) - logInfo("Completed getting MetaData in Shark Semantic Analysis") + logDebug("Completed getting MetaData in Shark Semantic Analysis") // Reset makes sure we don't run the mapred jobs generated by Hive. if (shouldReset) reset() @@ -247,7 +247,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with SharkSemanticAnalyzer.breakHivePlanByStages(terminalOpSeq) genMapRedTasks(qb, pctx, terminalOpSeq) - logInfo("Completed plan generation") + logDebug("Completed plan generation") } /** From f93216eef2dbd18a2e63c5687d39e02f0cff61f8 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 11:57:08 -0700 Subject: [PATCH 053/331] Skeleton for SparkDDLTask. A dependent/child task of Hive's DDLTask that executes Shark-specific metadata handlers. --- .../scala/shark/execution/SparkDDLTask.scala | 90 +++++++++++++++++++ 1 file changed, 90 insertions(+) create mode 100644 src/main/scala/shark/execution/SparkDDLTask.scala diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala new file mode 100644 index 00000000..2d3f4f29 --- /dev/null +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution + +import org.apache.hadoop.hive.ql.{Context, DriverContext} +import org.apache.hadoop.hive.ql.exec.{Task => HiveTask, TaskExecutionException} +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.plan._ +import org.apache.hadoop.hive.ql.plan.api.StageType + +import org.apache.spark.SparkEnv + +import shark.{LogHelper, SharkEnv} + + +class SparkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable + +class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelper { + + override def execute(driverContext: DriverContext): Int = { + val sparkEnv = SparkEnv.get + val hiveMetadataDb = Hive.get(conf) + + work.ddlDesc match { + case creatTblDesc: CreateTableDesc => { + createTable(sparkEnv, hiveMetadataDb, creatTblDesc) + } + case addPartitionDesc: AddPartitionDesc => { + addPartition(sparkEnv, hiveMetadataDb, addPartitionDesc) + } + case renamePartitionDesc: RenamePartitionDesc => { + renamePartition(sparkEnv, hiveMetadataDb, renamePartitionDesc) + } + case dropTableDesc: DropTableDesc => { + dropTable(sparkEnv, hiveMetadataDb, dropTableDesc) + } + case _ => { + throw new UnsupportedOperationException( + "Shark does not require specific DDL handling for: " + work.ddlDesc.getClass.getName) + } + } + + // Hive's task runner expects a '0' return value to indicate success and exceptions on + // failure. + return 0 + } + + def createTable(sparkEnv: SparkEnv, hiveMetadataDb: Hive, createTblDesc: CreateTableDesc): Int = { + return 0 + } + + def addPartition( + sparkEnv: SparkEnv, + hiveMetadataDb: Hive, + addPartitionDesc: AddPartitionDesc): Int = { + return 0 + } + + def renamePartition( + sparkEnv: SparkEnv, + hiveMetadataDb: Hive, + renamePartitionDesc: RenamePartitionDesc): Int = { + return 0 + } + + def dropTable(sparkEnv: SparkEnv, hiveMetadataDb: Hive, dropTableDesc: DropTableDesc): Int = { + return 0 + } + + override def getType = StageType.DDL + + override def getName = "DDL-SPARK" + + override def localizeMRTmpFilesImpl(ctx: Context) = Unit +} From f35290c4a9b7c9acb04eba73f12f0c0537292593 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 10:45:59 -0700 Subject: [PATCH 054/331] Add unit test names for cached, Hive-partitioned tables. --- src/test/scala/shark/SQLSuite.scala | 64 +++++++++++++++++++++++++---- 1 file changed, 56 insertions(+), 8 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 1d2190c4..644abfa7 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -250,14 +250,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } } - ignore("drop partition") { - sc.runSql("create table foo_cached(key int, val string) partitioned by (dt string)") - sc.runSql("insert overwrite table foo_cached partition(dt='100') select * from test") - expectSql("select count(*) from foo_cached", "500") - sc.runSql("alter table foo_cached drop partition(dt='100')") - expectSql("select count(*) from foo_cached", "0") - } - test("create cached table with 'shark.cache' flag in table properties") { sc.runSql("drop table if exists ctas_tbl_props") sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='true') as @@ -324,6 +316,62 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(!SharkEnv.memoryMetadataManager.contains("sharkTest5Cached")) } + ////////////////////////////////////////////////////////////////////////////// + // Caching Hive-partititioned tables + ////////////////////////////////////////////////////////////////////////////// + test("create cached, partitioned table using regular CREATE TABLE and '_cached' suffix") { + sc.runSql("drop table srcpart") + sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") + assert(SharkEnv.memoryMetadataManager.contains("srcpart")) + } + + test("create cached, partitioned table using regular CREATE TABLE and table properties") { + sc.runSql("drop table srcpart") + sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") + } + + test("drop cached, partitioned table that has a single partition") { + assert(true) + } + + test("drop cached, partitioned table that has multiple partitions") { + assert(true) + } + + test("drop cached, partitioned table that has a UnionRDD partition") { + assert(true) + } + + test("alter cached table by adding a new partition") { + sc.runSql("drop table srcpart") + sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") + sc.runSql("insert overwrite table srcpart_cached partition(keypart = 1) select * from test") + } + + test("alter cached table by dropping a partition") { + assert(true) + } + + test("insert into a partition of a cached table") { + assert(true) + } + + test("insert overwrite a partition of a cached table") { + assert(true) + } + + test("scan cached, partitioned table that's empty") { + assert(true) + } + + test("scan cached, partitioned table that has a single partition") { + assert(true) + } + + test("scan cached, partitioned table that has multiple partitions") { + assert(true) + } + ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// From 3802e272ad1f9187a2ec0200de5adc4a7d88d653 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 19 Sep 2013 21:10:12 -0700 Subject: [PATCH 055/331] SHARK-183: implementing BooleanBitSetCompression --- .../memstore2/column/ColumnBuilder.scala | 2 +- .../memstore2/column/ColumnIterator.scala | 1 + .../column/CompressedColumnIterator.scala | 35 +++++++++++ .../column/CompressionAlgorithm.scala | 63 +++++++++++++++++++ .../CompressedColumnIteratorSuite.scala | 16 +++++ .../column/CompressionAlgorithmSuite.scala | 30 +++++++++ 6 files changed, 146 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala index 4ca1f4e5..84988be3 100644 --- a/src/main/scala/shark/memstore2/column/ColumnBuilder.scala +++ b/src/main/scala/shark/memstore2/column/ColumnBuilder.scala @@ -145,7 +145,7 @@ object ColumnBuilder { case _ => new GenericColumnBuilder(columnOi) } if (shouldCompress) { - v.compressionSchemes = Seq(new RLE()) + v.compressionSchemes = Seq(new RLE(), new BooleanBitSetCompression()) } v } diff --git a/src/main/scala/shark/memstore2/column/ColumnIterator.scala b/src/main/scala/shark/memstore2/column/ColumnIterator.scala index 57fae71d..24924162 100644 --- a/src/main/scala/shark/memstore2/column/ColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/ColumnIterator.scala @@ -53,6 +53,7 @@ object Implicits { case DefaultCompressionType.typeID => DefaultCompressionType case RLECompressionType.typeID => RLECompressionType case DictionaryCompressionType.typeID => DictionaryCompressionType + case BooleanBitSetCompressionType.typeID => BooleanBitSetCompressionType case _ => throw new MemoryStoreException("Unknown compression type " + i) } diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index 5acd75a6..3cb5f441 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -19,6 +19,8 @@ package shark.memstore2.column import java.nio.ByteBuffer +import org.apache.hadoop.io.BooleanWritable + import shark.memstore2.column.Implicits._ /** @@ -41,6 +43,7 @@ trait CompressedColumnIterator extends ColumnIterator { case DefaultCompressionType => new DefaultDecoder(buffer, columnType) case RLECompressionType => new RLDecoder(buffer, columnType) case DictionaryCompressionType => new DictDecoder(buffer, columnType) + case BooleanBitSetCompressionType => new BooleanBitSetDecoder(buffer, columnType) case _ => throw new UnsupportedOperationException() } } @@ -123,3 +126,35 @@ class DictDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends I _dictionary(index) } } + +/** + * Boolean BitSet encoding. + */ +class BooleanBitSetDecoder[V]( + buffer: ByteBuffer, + columnType: ColumnType[_, V], + var _pos: Int, + var _uncompressedSize: Int, + var _curValue: Long, + var _writable: BooleanWritable + ) extends Iterator[V] { + + def this(buffer: ByteBuffer, columnType: ColumnType[_, V]) + = this(buffer, columnType, 0, buffer.getInt(), 0, new BooleanWritable()) + + override def hasNext = _pos < _uncompressedSize + + override def next(): V = { + val offset = _pos % BooleanBitSetCompression.BOOLEANS_PER_LONG + + if (offset == 0) { + _curValue = buffer.getLong() + } + + val retval: Boolean = (_curValue & (1 << offset)) != 0 + _pos += 1 + _writable.set(retval) + _writable.asInstanceOf[V] + } +} + diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index efab4bc5..1ebd5943 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -53,6 +53,8 @@ object RLECompressionType extends CompressionType(0) object DictionaryCompressionType extends CompressionType(1) +object BooleanBitSetCompressionType extends CompressionType(2) + /** * An no-op compression. */ @@ -277,3 +279,64 @@ class DictionaryEncoding extends CompressionAlgorithm { compressedBuffer } } + +/** +* BitSet compression for Boolean values. +*/ +object BooleanBitSetCompression { + val BOOLEANS_PER_LONG : Short = 64 +} + +class BooleanBitSetCompression extends CompressionAlgorithm { + + private var _uncompressedSize = 0 + + override def compressionType = BooleanBitSetCompressionType + + override def supportsType(t: ColumnType[_, _]) = { + t match { + case BOOLEAN => true + case _ => false + } + } + + override def gatherStatsForCompressibility[T](v: T, t: ColumnType[T,_]) { + val s = t.actualSize(v) + _uncompressedSize += s + } + + // Booleans are encoded into Longs; in addition, we need one int to store the number of + // Booleans contained in the compressed buffer. + override def compressedSize: Int = math.ceil(_uncompressedSize.toFloat / BooleanBitSetCompression.BOOLEANS_PER_LONG).toInt * 8 + 4 + + override def uncompressedSize: Int = _uncompressedSize + + override def compress[T](b: ByteBuffer, t: ColumnType[T,_]): ByteBuffer = { + // Leave 4 extra bytes for column type, another 4 for compression type. + val compressedBuffer = ByteBuffer.allocate(4 + 4 + compressedSize) + compressedBuffer.order(ByteOrder.nativeOrder()) + compressedBuffer.putInt(b.getInt()) + compressedBuffer.putInt(compressionType.typeID) + compressedBuffer.putInt(b.remaining()) + + var cur: Long = 0 + var pos: Int = 0 + var offset: Int = 0 + + while (b.hasRemaining) { + offset = pos % BooleanBitSetCompression.BOOLEANS_PER_LONG + val elem = t.extract(b).asInstanceOf[Boolean] + + if (elem) { + cur = (cur | (1 << offset)).toLong + } + if (offset == BooleanBitSetCompression.BOOLEANS_PER_LONG - 1 || !b.hasRemaining) { + compressedBuffer.putLong(cur) + cur = 0 + } + pos += 1 + } + compressedBuffer.rewind() + compressedBuffer + } +} diff --git a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala index 3096ce28..6ed0aa4d 100644 --- a/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala @@ -153,6 +153,22 @@ class CompressedColumnIteratorSuite extends FunSuite { val expectedLen = (Short.MaxValue.toInt + 1) * (4 + 4) testList(ints, INT, new RLE, expectedLen, shouldNotCompress = true) } + + test("BooleanBitSet Boolean (shorter)") { + // 1 Long worth of Booleans, in addtion to the length field: 4+8 + val bools = Seq(true, true, false, false) + testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8) + } + + test("BooleanBitSet Boolean (longer)") { + // 2 Longs worth of Booleans, in addtion to the length field: 4+8+8 + val bools = Seq(true, true, false, false, true, true, false, false,true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false) + testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8+8) + } } diff --git a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala index 3059b40c..57eab675 100644 --- a/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ b/src/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -258,4 +258,34 @@ class CompressionAlgorithmSuite extends FunSuite { assert(newBuffer.getInt() === STRING.typeID) assert(newBuffer.getInt() === DefaultCompressionType.typeID) } + + test("BooleanBitSet encoding") { + val bbs = new BooleanBitSetCompression() + val b = ByteBuffer.allocate(4 + 64 + 2) + b.order(ByteOrder.nativeOrder()) + b.putInt(BOOLEAN.typeID) + for(_ <- 1 to 5) { + b.put(0.toByte) + b.put(1.toByte) + bbs.gatherStatsForCompressibility(false, BOOLEAN) + bbs.gatherStatsForCompressibility(true, BOOLEAN) + } + for(_ <- 1 to 54) { + b.put(0.toByte) + bbs.gatherStatsForCompressibility(false, BOOLEAN) + } + b.put(0.toByte) + b.put(1.toByte) + bbs.gatherStatsForCompressibility(false, BOOLEAN) + bbs.gatherStatsForCompressibility(true, BOOLEAN) + b.limit(b.position()) + b.rewind() + val compressedBuffer = bbs.compress(b, BOOLEAN) + assert(compressedBuffer.getInt() === BOOLEAN.typeID) + assert(compressedBuffer.getInt() === BooleanBitSetCompressionType.typeID) + assert(compressedBuffer.getInt() === 64 + 2) + assert(compressedBuffer.getLong() === 682) + assert(compressedBuffer.getLong() === 2) + assert(!compressedBuffer.hasRemaining) + } } From ce86bd11906871ad8c6fbb49dc9c48319f90ee4c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 13:16:36 -0700 Subject: [PATCH 056/331] Register SparkDDLWork and use it in SharkSemanticAnalyzer to support creating cached tables from CREATE TABLE commands. --- src/main/scala/shark/SharkDriver.scala | 4 +- .../shark/parse/SharkSemanticAnalyzer.scala | 87 +++++++++---------- src/test/scala/shark/SQLSuite.scala | 24 +++-- 3 files changed, 61 insertions(+), 54 deletions(-) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index d6c9687a..2c8f0a5b 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -35,7 +35,8 @@ import org.apache.hadoop.util.StringUtils import shark.api.TableRDD import shark.api.QueryExecutionException -import shark.execution.{SharkExplainTask, SharkExplainWork, SparkTask, SparkWork} +import shark.execution.{SharkExplainTask, SharkExplainWork, SparkDDLTask, SparkDDLWork, SparkTask, + SparkWork} import shark.memstore2.ColumnarSerDe import shark.parse.{QueryContext, SharkSemanticAnalyzerFactory} @@ -62,6 +63,7 @@ private[shark] object SharkDriver extends LogHelper { // Task factory. Add Shark specific tasks. TaskFactory.taskvec.addAll(Seq( + new TaskFactory.taskTuple(classOf[SparkDDLWork], classOf[SparkDDLTask]), new TaskFactory.taskTuple(classOf[SparkWork], classOf[SparkTask]), new TaskFactory.taskTuple(classOf[SharkExplainWork], classOf[SharkExplainTask]))) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 2f05bb1d..989f7fc5 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -40,7 +40,7 @@ import org.apache.spark.storage.StorageLevel import shark.{CachedTableRecovery, LogHelper, SharkConfVars, SharkEnv, Utils} import shark.execution.{HiveOperator, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator, - SparkWork, TerminalOperator} + SparkDDLWork, SparkWork, TerminalOperator} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} @@ -89,20 +89,16 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with super.analyzeInternal(ast) // Do post-Hive analysis of the CREATE TABLE (e.g detect caching mode). analyzeCreateTable(ast, qb) match { - case Some(selectStmtASTNode) => { + case Some(queryStmtASTNode) => { // Set the 'child' to reference the SELECT statement root node, with is a // HiveParer.HIVE_QUERY. - child = selectStmtASTNode - // Hive's super.analyzeInternal() generates MapReduce tasks for the SELECT. Avoid - // executing those tasks by reset()-ing some Hive SemanticAnalyzer state after phase 1 of - // analysis below. - // TODO(harvey): This might be too much. SharkSemanticAnalyzer could just clear - // 'rootTasks', since it's a protected field. + child = queryStmtASTNode + // Hive's super.analyzeInternal() might generate MapReduce tasks. Avoid executing those + // tasks by reset()-ing some Hive SemanticAnalyzer state after doPhase1(). shouldReset = true } case None => { - // Done with semantic analysis if the CREATE TABLE statement isn't a CTAS. The DDLTask - // created from 'super.analyzeInternal()' will be used to create the table. + // Done with semantic analysis if the CREATE TABLE statement isn't a CTAS. return } } @@ -332,43 +328,44 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } } - def analyzeCreateTable(rootAST: ASTNode, queryBlock: QueryBlock): Option[ASTNode] = { - // If we detect that the CREATE TABLE is part of a CTAS, then this is set to the root node of the - // SELECT statement - var selectStmtASTNode: Option[ASTNode] = None + // If we detect that the CREATE TABLE is part of a CTAS, then this is set to the root node of + // the query command (i.e., the root node of the SELECT statement). + var queryStmtASTNode: Option[ASTNode] = None // TODO(harvey): We might be able to reuse the QB passed into this method, as long as it was // created after the super.analyzeInternal() call. That QB and the createTableDesc // should have everything (e.g. isCTAS(), partCols). Note that the QB might not be // accessible from getParseContext(), since the SemanticAnalyzer#analyzeInternal() // doesn't set (this.qb = qb) for a non-CTAS. - var isCTAS = false + var isRegularCreateTable = true var isHivePartitioned = false for (ch <- rootAST.getChildren) { ch.asInstanceOf[ASTNode].getToken.getType match { case HiveParser.TOK_QUERY => { - isCTAS = true - selectStmtASTNode = Some(ch.asInstanceOf[ASTNode]) - } - case HiveParser.TOK_TABLEPARTCOLS => { - isHivePartitioned = true + isRegularCreateTable = false + queryStmtASTNode = Some(ch.asInstanceOf[ASTNode]) } case _ => Unit } } - // The 'createTableDesc' can be NULL if the command is a ... - // 1) syntactically valid CREATE TABLE statement. Note that the table specified may or may not - // already exist. If the table already exists, then an exception is thrown by the DDLTask - // that's executed after semantic analysis. - // 2) valid CTAS statement with an IF NOT EXISTS condition and the specified table already - // exists. If the table to-be-created already exists, and the CTAS statement does not - // have an IF NOT EXISTS condition, then an exception will be thrown by - // SemanticAnalzyer#analyzeInternal(). - val createTableDesc = getParseContext.getQB.getTableDesc - if (isCTAS && createTableDesc != null) { + // Invariant: At this point, the command is either a CTAS or a CREATE TABLE. + var ddlTasks: Seq[DDLTask] = Nil + val createTableDesc = + if (isRegularCreateTable) { + // Unfortunately, we have to comb the root tasks because for CREATE TABLE, + // SemanticAnalyzer#analyzeCreateTable() does't set the CreateTableDesc in its QB. + ddlTasks = rootTasks.filter(_.isInstanceOf[DDLTask]).asInstanceOf[Seq[DDLTask]] + if (ddlTasks.isEmpty) null else ddlTasks.head.getWork.getCreateTblDesc + } + else { + getParseContext.getQB.getTableDesc + } + + // Can be NULL if there is an IF NOTE EXISTS condition and the table already exists. + if (createTableDesc != null) { val tableName = createTableDesc.getTableName val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) // The CreateTableDesc's table properties are Java Maps, but the TableDesc's table properties, @@ -390,30 +387,26 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with createTableProperties.put("shark.cache", cacheMode.toString) } + // For CTAS, the SparkTask's MemoryStoreSinkOperator will create the table and the Hive + // DDLTask will be a dependent of the SparkTask. SparkTasks are created in genMapRedTasks(). + if (isRegularCreateTable) { + // In Hive, a CREATE TABLE command is handled by a DDLTask, created by + // SemanticAnalyzer#analyzeCreateTable(). The DDL tasks' execution succeeds only if the + // CREATE TABLE is valid. So, hook a SharkDDLTask as a dependent of the Hive DDLTask so that + // Shark metadata is updated only if the Hive task execution is successful. + val hiveDDLTask = ddlTasks.head; + val sharkDDLWork = new SparkDDLWork(createTableDesc) + hiveDDLTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) + } + if (CacheType.shouldCache(cacheMode)) { createTableDesc.setSerName(classOf[ColumnarSerDe].getName) - - if (isHivePartitioned) { - // TODO(harvey): Remove once it's supported ... - throw new SemanticException( - "Support for cached, Hive-partitioned tables coming soon!") - } - - // In Hive, a CREATE TABLE command is handled by a DDLTask, which in this case, is created - // by the Hive SemanticAnalyzer's genMapRedTasks and not Hive's DDLSemanticAnalyzer. Since - // creating tables in Shark doesn't involve too much overhead (we don't support features - // such as indexing), just directly update the Shark MemoryMetaDataManager in this method. - - // Make sure that the table exists. - // TODO(harvey): This might have to go in a SparkDDLTask wrapper. - // SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned) } queryBlock.setCacheModeForCreateTable(cacheMode) queryBlock.setTableDesc(createTableDesc) } - - return selectStmtASTNode + return queryStmtASTNode } } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 644abfa7..a093096e 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -320,55 +320,67 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Caching Hive-partititioned tables ////////////////////////////////////////////////////////////////////////////// test("create cached, partitioned table using regular CREATE TABLE and '_cached' suffix") { - sc.runSql("drop table srcpart") + sc.runSql("drop table if exists srcpart_cached") sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") - assert(SharkEnv.memoryMetadataManager.contains("srcpart")) + // assert(SharkEnv.memoryMetadataManager.contains("srcpart")) + assert(true) } test("create cached, partitioned table using regular CREATE TABLE and table properties") { - sc.runSql("drop table srcpart") + sc.runSql("drop table if exists srcpart_cached") sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") + assert(true) } test("drop cached, partitioned table that has a single partition") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("drop cached, partitioned table that has multiple partitions") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("drop cached, partitioned table that has a UnionRDD partition") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("alter cached table by adding a new partition") { - sc.runSql("drop table srcpart") - sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") - sc.runSql("insert overwrite table srcpart_cached partition(keypart = 1) select * from test") + sc.runSql("drop table if exists srcpart_cached") + // sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") + // sc.runSql("insert overwrite table srcpart_cached partition(keypart = 1) select * from test") + assert(true) } test("alter cached table by dropping a partition") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("insert into a partition of a cached table") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("insert overwrite a partition of a cached table") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("scan cached, partitioned table that's empty") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("scan cached, partitioned table that has a single partition") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } test("scan cached, partitioned table that has multiple partitions") { + sc.runSql("drop table if exists srcpart_cached") assert(true) } From f8dd14b1a8fc2aae7708aea1284b3944f40c5ae6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 14:40:04 -0700 Subject: [PATCH 057/331] Track and record CacheType for CREATE TABLE and CTAS. --- .../execution/MemoryStoreSinkOperator.scala | 8 ++--- .../shark/execution/OperatorFactory.scala | 5 +-- .../scala/shark/execution/SparkDDLTask.scala | 27 ++++++++-------- .../memstore2/MemoryMetadataManager.scala | 32 +++++++++++++------ .../scala/shark/memstore2/MemoryTable.scala | 12 ++++++- .../parse/SharkDDLSemanticAnalyzer.scala | 19 +++++++++-- .../shark/parse/SharkSemanticAnalyzer.scala | 14 +++++--- 7 files changed, 80 insertions(+), 37 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index fc989885..6d7013e2 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -42,7 +42,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { @BeanProperty var shouldCompress: Boolean = _ @BeanProperty var storageLevel: StorageLevel = _ @BeanProperty var tableName: String = _ - @transient var useTachyon: Boolean = _ + @transient var cacheMode: CacheType.CacheType = _ @transient var useUnionRDD: Boolean = _ @transient var numColumns: Int = _ @@ -65,7 +65,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { val op = OperatorSerializationWrapper(this) val tachyonWriter: TachyonTableWriter = - if (useTachyon) { + if (cacheMode == CacheType.TACHYON) { // Use an additional row to store metadata (e.g. number of rows in each partition). SharkEnv.tachyonUtil.createTableWriter(tableName, numColumns + 1) } else { @@ -100,7 +100,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Put the table in Tachyon. op.logInfo("Putting RDD for %s in Tachyon".format(tableName)) - SharkEnv.memoryMetadataManager.put(tableName, rdd) + SharkEnv.memoryMetadataManager.put(tableName, rdd, cacheMode) tachyonWriter.createTable(ByteBuffer.allocate(0)) rdd = rdd.mapPartitionsWithIndex { case(partitionIndex, iter) => @@ -133,7 +133,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { rdd = rdd.union( SharkEnv.memoryMetadataManager.get(tableName).get.asInstanceOf[RDD[TablePartition]]) } - SharkEnv.memoryMetadataManager.put(tableName, rdd) + SharkEnv.memoryMetadataManager.put(tableName, rdd, cacheMode) rdd.setName(tableName) // Run a job on the original RDD to force it to go into cache. diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 97a6851a..9fa8b4d7 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.spark.storage.StorageLevel import shark.LogHelper +import shark.memstore2.CacheType._ /** @@ -47,14 +48,14 @@ object OperatorFactory extends LogHelper { tableName: String, storageLevel: StorageLevel, numColumns: Int, - useTachyon: Boolean, + cacheMode: CacheType, useUnionRDD: Boolean): TerminalOperator = { val sinkOp = _newOperatorInstance( classOf[MemoryStoreSinkOperator], hiveTerminalOp).asInstanceOf[MemoryStoreSinkOperator] sinkOp.tableName = tableName sinkOp.storageLevel = storageLevel sinkOp.numColumns = numColumns - sinkOp.useTachyon = useTachyon + sinkOp.cacheMode = cacheMode sinkOp.useUnionRDD = useUnionRDD _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 2d3f4f29..8136ffd7 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -26,11 +26,15 @@ import org.apache.hadoop.hive.ql.plan.api.StageType import org.apache.spark.SparkEnv import shark.{LogHelper, SharkEnv} +import shark.memstore2.CacheType -class SparkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable +private[shark] class SparkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { + // Used only for CREATE TABLE. + var cacheMode: CacheType.CacheType = _ +} -class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelper { +private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelper { override def execute(driverContext: DriverContext): Int = { val sparkEnv = SparkEnv.get @@ -38,14 +42,11 @@ class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelp work.ddlDesc match { case creatTblDesc: CreateTableDesc => { - createTable(sparkEnv, hiveMetadataDb, creatTblDesc) + createTable(sparkEnv, hiveMetadataDb, creatTblDesc, work.cacheMode) } case addPartitionDesc: AddPartitionDesc => { addPartition(sparkEnv, hiveMetadataDb, addPartitionDesc) } - case renamePartitionDesc: RenamePartitionDesc => { - renamePartition(sparkEnv, hiveMetadataDb, renamePartitionDesc) - } case dropTableDesc: DropTableDesc => { dropTable(sparkEnv, hiveMetadataDb, dropTableDesc) } @@ -60,7 +61,11 @@ class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelp return 0 } - def createTable(sparkEnv: SparkEnv, hiveMetadataDb: Hive, createTblDesc: CreateTableDesc): Int = { + def createTable( + sparkEnv: SparkEnv, + hiveMetadataDb: Hive, + createTblDesc: CreateTableDesc, + cacheMode: CacheType.CacheType): Int = { return 0 } @@ -71,14 +76,10 @@ class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelp return 0 } - def renamePartition( + def dropTable( sparkEnv: SparkEnv, hiveMetadataDb: Hive, - renamePartitionDesc: RenamePartitionDesc): Int = { - return 0 - } - - def dropTable(sparkEnv: SparkEnv, hiveMetadataDb: Hive, dropTableDesc: DropTableDesc): Int = { + dropTableDesc: DropTableDesc): Int = { return 0 } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 53835e89..1d4deb3d 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -27,6 +27,7 @@ import org.apache.spark.storage.StorageLevel import shark.SharkConfVars + // TODO(harvey): Re-evaluate the interfaces to this class. For example, add() could be renamed to // addCreatedTable(). class MemoryMetadataManager { @@ -40,6 +41,13 @@ class MemoryMetadataManager { def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) + def getCacheMode(key: String): CacheType.CacheType = { + _keyToMemoryTable.get(key.toLowerCase) match { + case Some(memoryTable) => return memoryTable.cacheMode + case _ => return CacheType.NONE + } + } + def isHivePartitioned(key: String): Boolean = { _keyToMemoryTable.get(key.toLowerCase) match { case Some(memoryTable) => return memoryTable.isHivePartitioned @@ -47,24 +55,30 @@ class MemoryMetadataManager { } } - def add(key: String, isHivePartitioned: Boolean) { - _keyToMemoryTable.put(key.toLowerCase, new MemoryTable(key, isHivePartitioned)) + def add(key: String, isHivePartitioned: Boolean, cacheMode: CacheType.CacheType) { + val memoryTable = new MemoryTable(key, isHivePartitioned) + _keyToMemoryTable.put(key.toLowerCase, memoryTable) + memoryTable.cacheMode = cacheMode } - def put(key: String, rdd: RDD[_]) { + def put(key: String, rdd: RDD[_], cacheMode: CacheType.CacheType) { if (!_keyToMemoryTable.contains(key.toLowerCase)) { // TODO(harvey): Remove this once CREATE TABLE/CTAS handling involves calling add(). For now, // CTAS result caching is done by MemoryStoreSinkOperator, which calls this // put() method. - add(key, false /* isHivePartitioned */) + add(key, false /* isHivePartitioned */, cacheMode) } _keyToMemoryTable(key.toLowerCase).tableRDD = rdd } - def putHivePartition(key: String, partitionColumn: String, rdd: RDD[_]) { + def putHivePartition( + key: String, + partitionColumn: String, + rdd: RDD[_], + cacheMode: CacheType.CacheType) { if (!_keyToMemoryTable.contains(key.toLowerCase)) { // TODO(harvey): See comment for put() above. - add(key, true /* isHivePartitioned */) + add(key, true /* isHivePartitioned */, cacheMode) } _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumn) = rdd } @@ -99,9 +113,9 @@ class MemoryMetadataManager { * as well. * * @param key Name of the table to drop. - * @return Option::isEmpty() is true of there is no MemoryTable corresponding to 'key' in - * _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will be - * a UnionRDD comprising RDDs that represent the Hive-partitions. + * @return Option::isEmpty() is true of there is no MemoryTable (and RDD) corresponding to 'key' + * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will + * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ def unpersist(key: String): Option[RDD[_]] = { def unpersistRDD(rdd: RDD[_]) { diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 60e1360a..9c87d35a 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -36,7 +36,9 @@ import org.apache.spark.rdd.RDD * such as HivePartitionedTable or TachyonTable, subclass it. For now, there isn't * too much metadata to track, so it should be okay to have a single MemoryTable. */ -private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: Boolean) { +private[shark] class MemoryTable( + val tableName: String, + val isHivePartitioned: Boolean) { // Should only be used if the table is not Hive-partitioned. _tableRDD.isEmpty() is true if the // table does not contain any data (e.g. it was created from a CREATE TABLE command, but never @@ -47,6 +49,14 @@ private[shark] class MemoryTable(val tableName: String, val isHivePartitioned: B private val _hivePartitionRDDs: Map[String, RDD[_]] = if (isHivePartitioned) { new JavaHashMap[String, RDD[_]]() } else { null } + private var _cacheMode: CacheType.CacheType = CacheType.NONE + + def cacheMode = _cacheMode + + def cacheMode_= (value: CacheType.CacheType) { + _cacheMode = value + } + def tableRDD: Option[RDD[_]] = { assert ( !isHivePartitioned, diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index a43a4975..7a16f443 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -12,9 +12,22 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) override def analyzeInternal(node: ASTNode): Unit = { super.analyzeInternal(node) - //handle drop table query - if (node.getToken().getType() == HiveParser.TOK_DROPTABLE) { - SharkEnv.unpersist(getTableName(node)) + + node.getToken.getType match { + case HiveParser.TOK_DROPTABLE => { + SharkEnv.unpersist(getTableName(node)) + } + // Handle ALTER TABLE for cached, Hive-partitioned tables + case HiveParser.TOK_ALTERTABLE_ADDPARTS => { + Unit + } + case HiveParser.TOK_ALTERTABLE_DROPPARTS => { + Unit + } + case HiveParser.TOK_ALTERTABLE_PARTITION => { + Unit + } + case _ => Unit } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 989f7fc5..3cc646ab 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -150,8 +150,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // TODO: clean the following code. It's too messy to understand... val terminalOpSeq = { - if (qb.getParseInfo.isInsertToTable && !qb.isCTAS) { - // Handle an INSERT into a cached table. + val qbParseInfo = qb.getParseInfo + if (qbParseInfo.isInsertToTable && !qb.isCTAS) { + // Handle INSERT. There can be multiple Hive sink operators if the single command comprises + // multiple INSERTs. hiveSinkOps.map { hiveSinkOp => val tableName = hiveSinkOp.asInstanceOf[HiveFileSinkOperator].getConf().getTableInfo() .getTableName() @@ -166,14 +168,15 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with case Some(rdd) => { if (hiveSinkOps.size == 1) { // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. - val useUnionRDD = qb.getParseInfo.isInsertIntoTable(cachedTableName) + val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) val storageLevel = RDDUtils.getStorageLevelOfCachedTable(rdd) + val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, cachedTableName, storageLevel, _resSchema.size, // numColumns - qb.getCacheModeForCreateTable == CacheType.TACHYON, // use tachyon + cacheMode, useUnionRDD) } else { throw new SemanticException( @@ -198,7 +201,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.getTableDesc.getTableName, storageLevel, _resSchema.size, // numColumns - qb.getCacheModeForCreateTable == CacheType.TACHYON, // use tachyon + qb.getCacheModeForCreateTable, false) } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) @@ -396,6 +399,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // Shark metadata is updated only if the Hive task execution is successful. val hiveDDLTask = ddlTasks.head; val sharkDDLWork = new SparkDDLWork(createTableDesc) + sharkDDLWork.cacheMode = cacheMode hiveDDLTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } From 39cfe440775d467c8b9d50d0aafe4d0a49dff1af Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 14:50:41 -0700 Subject: [PATCH 058/331] Unit test names for creating cached tables using regular CREATE TABLE. --- src/test/scala/shark/SQLSuite.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index a093096e..fb41c799 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -319,14 +319,22 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // Caching Hive-partititioned tables ////////////////////////////////////////////////////////////////////////////// - test("create cached, partitioned table using regular CREATE TABLE and '_cached' suffix") { + test("Use regular CREATE TABLE to create cached table") { + assert(true) + } + + test("Insert into empty cached table") { + assert(true) + } + + test("Use regular CREATE TABLE and '_cached' suffix to create cached, partitioned table") { sc.runSql("drop table if exists srcpart_cached") sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") // assert(SharkEnv.memoryMetadataManager.contains("srcpart")) assert(true) } - test("create cached, partitioned table using regular CREATE TABLE and table properties") { + test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { sc.runSql("drop table if exists srcpart_cached") sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") assert(true) From 94d91803caa7d89d972e73e25cbc6f950e498693 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 15:00:57 -0700 Subject: [PATCH 059/331] Fix some nits before enabling a ton of cached table features. --- .../scala/shark/execution/MemoryStoreSinkOperator.scala | 1 + .../scala/shark/memstore2/MemoryMetadataManager.scala | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 6d7013e2..fc16e751 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -42,6 +42,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { @BeanProperty var shouldCompress: Boolean = _ @BeanProperty var storageLevel: StorageLevel = _ @BeanProperty var tableName: String = _ + @BeanProperty var partitionColumnValues: String = _ // delimited by '/' @transient var cacheMode: CacheType.CacheType = _ @transient var useUnionRDD: Boolean = _ @transient var numColumns: Int = _ diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 1d4deb3d..5cd4fef5 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -73,14 +73,14 @@ class MemoryMetadataManager { def putHivePartition( key: String, - partitionColumn: String, + partitionColumnValues: String, rdd: RDD[_], cacheMode: CacheType.CacheType) { if (!_keyToMemoryTable.contains(key.toLowerCase)) { // TODO(harvey): See comment for put() above. add(key, true /* isHivePartitioned */, cacheMode) } - _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumn) = rdd + _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumnValues) = rdd } def get(key: String): Option[RDD[_]] = { @@ -88,8 +88,8 @@ class MemoryMetadataManager { return memoryTableValue.flatMap(_.tableRDD) } - def getHivePartition(key: String, partitionColumn: String): Option[RDD[_]] = { - return _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs.get(partitionColumn) + def getHivePartition(key: String, partitionColumnValues: String): Option[RDD[_]] = { + return _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs.get(partitionColumnValues) } def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { From b0ee377e410c47502ba33a278cd4559bde6bcb2c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 15:45:44 -0700 Subject: [PATCH 060/331] Remove SparkEnv from SparkDDLTask. Don't use it to compute table identifiers (yet). --- .../scala/shark/execution/SparkDDLTask.scala | 25 ++++++++----------- .../memstore2/MemoryMetadataManager.scala | 4 +-- 2 files changed, 12 insertions(+), 17 deletions(-) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 8136ffd7..7b17b5fe 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -23,8 +23,6 @@ import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.plan.api.StageType -import org.apache.spark.SparkEnv - import shark.{LogHelper, SharkEnv} import shark.memstore2.CacheType @@ -37,18 +35,17 @@ private[shark] class SparkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelper { override def execute(driverContext: DriverContext): Int = { - val sparkEnv = SparkEnv.get val hiveMetadataDb = Hive.get(conf) work.ddlDesc match { case creatTblDesc: CreateTableDesc => { - createTable(sparkEnv, hiveMetadataDb, creatTblDesc, work.cacheMode) + createTable(hiveMetadataDb, creatTblDesc, work.cacheMode) } case addPartitionDesc: AddPartitionDesc => { - addPartition(sparkEnv, hiveMetadataDb, addPartitionDesc) + addPartition(hiveMetadataDb, addPartitionDesc) } case dropTableDesc: DropTableDesc => { - dropTable(sparkEnv, hiveMetadataDb, dropTableDesc) + dropTable(hiveMetadataDb, dropTableDesc) } case _ => { throw new UnsupportedOperationException( @@ -62,25 +59,23 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab } def createTable( - sparkEnv: SparkEnv, hiveMetadataDb: Hive, createTblDesc: CreateTableDesc, - cacheMode: CacheType.CacheType): Int = { - return 0 + cacheMode: CacheType.CacheType) { + val isHivePartitioned = (createTblDesc.getPartCols.size > 0) + SharkEnv.memoryMetadataManager.add(createTblDesc.getTableName, isHivePartitioned, cacheMode) } def addPartition( - sparkEnv: SparkEnv, hiveMetadataDb: Hive, - addPartitionDesc: AddPartitionDesc): Int = { - return 0 + addPartitionDesc: AddPartitionDesc) { + Unit } def dropTable( - sparkEnv: SparkEnv, hiveMetadataDb: Hive, - dropTableDesc: DropTableDesc): Int = { - return 0 + dropTableDesc: DropTableDesc) { + Unit } override def getType = StageType.DDL diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 5cd4fef5..d878c654 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -39,8 +39,6 @@ class MemoryMetadataManager { private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) - def getCacheMode(key: String): CacheType.CacheType = { _keyToMemoryTable.get(key.toLowerCase) match { case Some(memoryTable) => return memoryTable.cacheMode @@ -55,6 +53,8 @@ class MemoryMetadataManager { } } + def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) + def add(key: String, isHivePartitioned: Boolean, cacheMode: CacheType.CacheType) { val memoryTable = new MemoryTable(key, isHivePartitioned) _keyToMemoryTable.put(key.toLowerCase, memoryTable) From ac77a4e719b7dc7604140b496f93383f5f61d1c6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 15:45:57 -0700 Subject: [PATCH 061/331] Unit tests for creating cached tables using CREATE TABLE. --- .../memstore2/MemoryMetadataManager.scala | 2 +- .../scala/shark/memstore2/MemoryTable.scala | 3 +- src/test/scala/shark/SQLSuite.scala | 38 +++++++++++++------ 3 files changed, 30 insertions(+), 13 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index d878c654..3dd533f1 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -136,7 +136,7 @@ class MemoryMetadataManager { val hivePartitionRDDs = memoryTable.hivePartitionRDDs.values.toSeq.asInstanceOf[Seq[RDD[Any]]] if (hivePartitionRDDs.size > 0) { - return Some(new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs)) + return Some(unpersistRDD(new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs))) } return None } else { diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 9c87d35a..80ed5629 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -36,7 +36,8 @@ import org.apache.spark.rdd.RDD * such as HivePartitionedTable or TachyonTable, subclass it. For now, there isn't * too much metadata to track, so it should be okay to have a single MemoryTable. */ -private[shark] class MemoryTable( +private[shark] +class MemoryTable( val tableName: String, val isHivePartitioned: Boolean) { diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index fb41c799..47c7c153 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -319,8 +319,21 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // Caching Hive-partititioned tables ////////////////////////////////////////////////////////////////////////////// - test("Use regular CREATE TABLE to create cached table") { - assert(true) + test("Use regular CREATE TABLE and '_cached' suffix to create cached table") { + sc.runSql("drop table if exists empty_table_cached") + sc.runSql("create table empty_table_cached(key string, value string)") + assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached")) + sc.runSql("drop table if exists empty_table_cached") + } + + test("Use regular CREATE TABLE and table properties to create cached table") { + sc.runSql("drop table if exists empty_table_cached_tbl_props") + sc.runSql("""create table empty_table_cached_tbl_props(key string, value string) + TBLPROPERTIES('shark.cache' = 'true')""") + assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached_tbl_props")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached_tbl_props")) + sc.runSql("drop table if exists empty_table_cached_tbl_props") } test("Insert into empty cached table") { @@ -328,16 +341,21 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("Use regular CREATE TABLE and '_cached' suffix to create cached, partitioned table") { - sc.runSql("drop table if exists srcpart_cached") - sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") - // assert(SharkEnv.memoryMetadataManager.contains("srcpart")) - assert(true) + sc.runSql("drop table if exists empty_part_table_cached") + sc.runSql("""create table empty_part_table_cached(key int, val string) + partitioned by (keypart int)""") + assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached")) + assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached")) + sc.runSql("drop table if exists empty_part_table_cached") } test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { - sc.runSql("drop table if exists srcpart_cached") - sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") - assert(true) + sc.runSql("drop table if exists empty_part_table_cached_tbl_props") + sc.runSql("""create table empty_part_table_cached_tbl_props(key int, val string) + partitioned by (keypart int)""") + assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) + assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached_tbl_props")) + sc.runSql("drop table if exists empty_part_table_cached_tbl_props") } test("drop cached, partitioned table that has a single partition") { @@ -357,8 +375,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("alter cached table by adding a new partition") { sc.runSql("drop table if exists srcpart_cached") - // sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") - // sc.runSql("insert overwrite table srcpart_cached partition(keypart = 1) select * from test") assert(true) } From 28ac4c2f51b0a2051588d687e1bc7af4aee86cf3 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 17:51:24 -0700 Subject: [PATCH 062/331] Get inserts to work on empty table. Fix some bugs that broke inserts on Hive tables... --- .../execution/MemoryStoreSinkOperator.scala | 26 ++++-- src/main/scala/shark/execution/RDDUtils.scala | 4 +- .../scala/shark/execution/SparkDDLTask.scala | 6 +- .../shark/execution/TableScanOperator.scala | 1 + .../memstore2/MemoryMetadataManager.scala | 89 ++++++++++--------- .../scala/shark/memstore2/MemoryTable.scala | 45 ++-------- .../shark/parse/SharkSemanticAnalyzer.scala | 13 +-- src/test/scala/shark/SQLSuite.scala | 76 ++++++++-------- 8 files changed, 130 insertions(+), 130 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index fc16e751..6a616ee5 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -42,7 +42,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { @BeanProperty var shouldCompress: Boolean = _ @BeanProperty var storageLevel: StorageLevel = _ @BeanProperty var tableName: String = _ - @BeanProperty var partitionColumnValues: String = _ // delimited by '/' + @BeanProperty var partitionColumnValues: Option[String] = None // delimited by '/' @transient var cacheMode: CacheType.CacheType = _ @transient var useUnionRDD: Boolean = _ @transient var numColumns: Int = _ @@ -100,9 +100,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { if (tachyonWriter != null) { // Put the table in Tachyon. op.logInfo("Putting RDD for %s in Tachyon".format(tableName)) - - SharkEnv.memoryMetadataManager.put(tableName, rdd, cacheMode) - tachyonWriter.createTable(ByteBuffer.allocate(0)) rdd = rdd.mapPartitionsWithIndex { case(partitionIndex, iter) => val partition = iter.next() @@ -134,13 +131,22 @@ class MemoryStoreSinkOperator extends TerminalOperator { rdd = rdd.union( SharkEnv.memoryMetadataManager.get(tableName).get.asInstanceOf[RDD[TablePartition]]) } - SharkEnv.memoryMetadataManager.put(tableName, rdd, cacheMode) rdd.setName(tableName) // Run a job on the original RDD to force it to go into cache. origRdd.context.runJob(origRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } + if (!SharkEnv.memoryMetadataManager.contains(tableName)) { + SharkEnv.memoryMetadataManager.add(tableName, partitionColumnValues.isDefined) + } + if (SharkEnv.memoryMetadataManager.isHivePartitioned(tableName)) { + SharkEnv.memoryMetadataManager.putHivePartition( + tableName, partitionColumnValues.get, rdd, cacheMode) + } else { + SharkEnv.memoryMetadataManager.put(tableName, rdd, cacheMode) + } + // Report remaining memory. /* Commented out for now waiting for the reporting code to make into Spark. val remainingMems: Map[String, (Long, Long)] = SharkEnv.sc.getSlavesMemoryStatus @@ -160,9 +166,13 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Combine stats for the two tables being combined. val numPartitions = statsAcc.value.toMap.size val currentStats = statsAcc.value - val otherIndexToStats = SharkEnv.memoryMetadataManager.getStats(tableName).get - for ((otherIndex, tableStats) <- otherIndexToStats) { - currentStats.append((otherIndex + numPartitions, tableStats)) + SharkEnv.memoryMetadataManager.getStats(tableName) match { + case Some(otherIndexToStats) => { + for ((otherIndex, tableStats) <- otherIndexToStats) { + currentStats.append((otherIndex + numPartitions, tableStats)) + } + } + case _ => Unit } currentStats.toMap } else { diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index bd68890b..2c52840f 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -34,12 +34,12 @@ import shark.SharkEnv */ object RDDUtils { - def getStorageLevelOfCachedTable(rdd: RDD[_]): StorageLevel = { + def getStorageLevelOfCachedRDD(rdd: RDD[_]): StorageLevel = { rdd match { case u: UnionRDD[_] => u.rdds.foldLeft(rdd.getStorageLevel) { (s, r) => { if (s == StorageLevel.NONE) { - getStorageLevelOfCachedTable(r) + getStorageLevelOfCachedRDD(r) } else { s } diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 7b17b5fe..bad62730 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -63,7 +63,11 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { val isHivePartitioned = (createTblDesc.getPartCols.size > 0) - SharkEnv.memoryMetadataManager.add(createTblDesc.getTableName, isHivePartitioned, cacheMode) + val tableName = createTblDesc.getTableName + SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned) + if (!isHivePartitioned) { + SharkEnv.memoryMetadataManager.put(tableName, new EmptyRDD(SharkEnv.sc), cacheMode) + } } def addPartition( diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 9115bac7..e82657e2 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -118,6 +118,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // 3. Hive table on HDFS (or other Hadoop storage) val cacheMode = CacheType.fromString( tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) + // TODO(harvey): Pruning Hive-partitioned, cached tables aren't supported yet. if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). val rdd = SharkEnv.memoryMetadataManager.get(tableKey).getOrElse { diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 3dd533f1..9544b974 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -18,6 +18,7 @@ package shark.memstore2 import java.util.concurrent.ConcurrentHashMap +import java.util.{HashMap => JavaHashMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ConcurrentMap @@ -26,10 +27,11 @@ import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.storage.StorageLevel import shark.SharkConfVars +import shark.SharkEnv -// TODO(harvey): Re-evaluate the interfaces to this class. For example, add() could be renamed to -// addCreatedTable(). +// TODO(harvey): Redo the interfaces to this class. For example, add() could be renamed to +// addCreatedTable(), which should also take in a Hive DB (metastore) name. class MemoryMetadataManager { private val _keyToMemoryTable: ConcurrentMap[String, MemoryTable] = @@ -39,13 +41,6 @@ class MemoryMetadataManager { private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - def getCacheMode(key: String): CacheType.CacheType = { - _keyToMemoryTable.get(key.toLowerCase) match { - case Some(memoryTable) => return memoryTable.cacheMode - case _ => return CacheType.NONE - } - } - def isHivePartitioned(key: String): Boolean = { _keyToMemoryTable.get(key.toLowerCase) match { case Some(memoryTable) => return memoryTable.isHivePartitioned @@ -55,20 +50,33 @@ class MemoryMetadataManager { def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) - def add(key: String, isHivePartitioned: Boolean, cacheMode: CacheType.CacheType) { - val memoryTable = new MemoryTable(key, isHivePartitioned) - _keyToMemoryTable.put(key.toLowerCase, memoryTable) - memoryTable.cacheMode = cacheMode + def add(key: String, isHivePartitioned: Boolean) { + val memoryTable = new MemoryTable(key.toLowerCase, isHivePartitioned) + if (isHivePartitioned) { + memoryTable.keyToHivePartitions = new JavaHashMap[String, RDD[_]]() + memoryTable.keyToCacheModes = new JavaHashMap[String, CacheType.CacheType]() + } + _keyToMemoryTable(key.toLowerCase) = memoryTable } - def put(key: String, rdd: RDD[_], cacheMode: CacheType.CacheType) { - if (!_keyToMemoryTable.contains(key.toLowerCase)) { - // TODO(harvey): Remove this once CREATE TABLE/CTAS handling involves calling add(). For now, - // CTAS result caching is done by MemoryStoreSinkOperator, which calls this - // put() method. - add(key, false /* isHivePartitioned */, cacheMode) + def getCacheMode(key: String): CacheType.CacheType = { + _keyToMemoryTable.get(key.toLowerCase) match { + case Some(memoryTable) => return memoryTable.cacheMode + case _ => return CacheType.NONE } - _keyToMemoryTable(key.toLowerCase).tableRDD = rdd + } + + def getPartitionCacheMode(key: String, partitionColumnValues: String): CacheType.CacheType = { + _keyToMemoryTable.get(key.toLowerCase) match { + case Some(memoryTable) => return memoryTable.keyToCacheModes(partitionColumnValues) + case _ => return CacheType.NONE + } + } + + def put(key: String, rdd: RDD[_], cacheMode: CacheType.CacheType) { + val memoryTable = _keyToMemoryTable(key.toLowerCase) + memoryTable.tableRDD = rdd + memoryTable.cacheMode = cacheMode } def putHivePartition( @@ -76,20 +84,19 @@ class MemoryMetadataManager { partitionColumnValues: String, rdd: RDD[_], cacheMode: CacheType.CacheType) { - if (!_keyToMemoryTable.contains(key.toLowerCase)) { - // TODO(harvey): See comment for put() above. - add(key, true /* isHivePartitioned */, cacheMode) - } - _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs(partitionColumnValues) = rdd + _keyToMemoryTable(key.toLowerCase).keyToHivePartitions(partitionColumnValues) = rdd + _keyToMemoryTable(key.toLowerCase).keyToCacheModes(partitionColumnValues) = cacheMode } def get(key: String): Option[RDD[_]] = { - val memoryTableValue: Option[MemoryTable] = _keyToMemoryTable.get(key.toLowerCase) - return memoryTableValue.flatMap(_.tableRDD) + _keyToMemoryTable.get(key.toLowerCase) match { + case Some(memoryTable) => return Some(memoryTable.tableRDD) + case _ => return None + } } def getHivePartition(key: String, partitionColumnValues: String): Option[RDD[_]] = { - return _keyToMemoryTable(key.toLowerCase).hivePartitionRDDs.get(partitionColumnValues) + return _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.get(partitionColumnValues) } def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { @@ -118,37 +125,37 @@ class MemoryMetadataManager { * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ def unpersist(key: String): Option[RDD[_]] = { - def unpersistRDD(rdd: RDD[_]) { + def unpersistRDD(rdd: RDD[_]): RDD[_] = { rdd match { case u: UnionRDD[_] => { // Recursively unpersist() all RDDs that compose the UnionRDD. u.unpersist() - u.rdds.foreach { - r => unpersistRDD(r) + u.rdds.map { + r => r.unpersist() } } case r => r.unpersist() } + return rdd } def unpersistMemoryTable(memoryTable: MemoryTable): Option[RDD[_]] = { + var unpersistedRDD: Option[RDD[_]] = None if (memoryTable.isHivePartitioned) { // unpersist() all RDDs for all Hive-partitions. - val hivePartitionRDDs = - memoryTable.hivePartitionRDDs.values.toSeq.asInstanceOf[Seq[RDD[Any]]] - if (hivePartitionRDDs.size > 0) { - return Some(unpersistRDD(new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs))) + val unpersistedRDDs = memoryTable.keyToHivePartitions.values.map( + rdd => unpersistRDD(rdd)).asInstanceOf[Seq[RDD[Any]]] + if (unpersistedRDDs.size > 0) { + val unionedRDD = new UnionRDD(unpersistedRDDs.head.context, unpersistedRDDs) + unpersistedRDD = Some(unionedRDD) } - return None } else { - if (memoryTable.tableRDD.isDefined) { - unpersistRDD(memoryTable.tableRDD.get) - } - return memoryTable.tableRDD + unpersistedRDD = Some(unpersistRDD(memoryTable.tableRDD)) } + return unpersistedRDD } // Remove MemoryTable's entry from Shark metadata. - _keyToStats.remove(key) + _keyToStats.remove(key.toLowerCase) val memoryTableValue: Option[MemoryTable] = _keyToMemoryTable.remove(key.toLowerCase) return memoryTableValue.flatMap(unpersistMemoryTable(_)) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 80ed5629..4f5e0480 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -41,44 +41,15 @@ class MemoryTable( val tableName: String, val isHivePartitioned: Boolean) { - // Should only be used if the table is not Hive-partitioned. _tableRDD.isEmpty() is true if the - // table does not contain any data (e.g. it was created from a CREATE TABLE command, but never - // populated with data). - private var _tableRDD: Option[RDD[_]] = None + // Should only be used if the table is not Hive-partitioned. + var tableRDD: RDD[_] = _ - // Should only be used if a cached table is Hive-partitioned. - private val _hivePartitionRDDs: Map[String, RDD[_]] = - if (isHivePartitioned) { new JavaHashMap[String, RDD[_]]() } else { null } - - private var _cacheMode: CacheType.CacheType = CacheType.NONE - - def cacheMode = _cacheMode + // CacheMode for the tableRDD. + var cacheMode: CacheType.CacheType = _ - def cacheMode_= (value: CacheType.CacheType) { - _cacheMode = value - } - - def tableRDD: Option[RDD[_]] = { - assert ( - !isHivePartitioned, - "Table " + tableName + " is Hive-partitioned. Use MemoryTableDesc::hivePartitionRDDs() " + - "to get RDDs corresponding to partition columns" - ) - return _tableRDD - } - - def tableRDD_= (value: RDD[_]) { - assert( - !isHivePartitioned, - "Table " + tableName + " is Hive-partitioned. Pass in a map of pairs " + - "the 'hivePartitionRDDs =' setter." - ) - _tableRDD = Some(value) - } + // Should only be used if a cached table is Hive-partitioned. + var keyToHivePartitions: Map[String, RDD[_]] = _ - def hivePartitionRDDs: Map[String, RDD[_]] = { - assert(isHivePartitioned, - "Table " + tableName + " is not Hive-partitioned. Use tableRDD() to get its RDD.") - _hivePartitionRDDs - } + // Map from Hive-Partition key to the cache mode for its RDD. + var keyToCacheModes: Map[String, CacheType.CacheType] = _ } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 3cc646ab..fb7b7e79 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -169,7 +169,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with if (hiveSinkOps.size == 1) { // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) - val storageLevel = RDDUtils.getStorageLevelOfCachedTable(rdd) + val storageLevel = RDDUtils.getStorageLevelOfCachedRDD(rdd) val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, @@ -390,9 +390,14 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with createTableProperties.put("shark.cache", cacheMode.toString) } + val shouldCache = CacheType.shouldCache(cacheMode) + if (shouldCache) { + createTableDesc.setSerName(classOf[ColumnarSerDe].getName) + } + // For CTAS, the SparkTask's MemoryStoreSinkOperator will create the table and the Hive // DDLTask will be a dependent of the SparkTask. SparkTasks are created in genMapRedTasks(). - if (isRegularCreateTable) { + if (isRegularCreateTable && shouldCache) { // In Hive, a CREATE TABLE command is handled by a DDLTask, created by // SemanticAnalyzer#analyzeCreateTable(). The DDL tasks' execution succeeds only if the // CREATE TABLE is valid. So, hook a SharkDDLTask as a dependent of the Hive DDLTask so that @@ -403,10 +408,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with hiveDDLTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } - if (CacheType.shouldCache(cacheMode)) { - createTableDesc.setSerName(classOf[ColumnarSerDe].getName) - } - queryBlock.setCacheModeForCreateTable(cacheMode) queryBlock.setTableDesc(createTableDesc) } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 47c7c153..84c6e64f 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -222,6 +222,28 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // cache DDL ////////////////////////////////////////////////////////////////////////////// + test("Use regular CREATE TABLE and '_cached' suffix to create cached table") { + sc.runSql("drop table if exists empty_table_cached") + sc.runSql("create table empty_table_cached(key string, value string)") + assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached")) + } + + test("Use regular CREATE TABLE and table properties to create cached table") { + sc.runSql("drop table if exists empty_table_cached_tbl_props") + sc.runSql("""create table empty_table_cached_tbl_props(key string, value string) + TBLPROPERTIES('shark.cache' = 'true')""") + assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached_tbl_props")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached_tbl_props")) + } + + test("Insert into empty cached table") { + sc.runSql("drop table if exists new_table_cached") + sc.runSql("create table new_table_cached(key string, value string)") + sc.runSql("insert into table new_table_cached select * from test where key > -1 limit 499") + expectSql("select count(*) from new_table_cached", "499") + } + test("insert into cached tables") { sc.runSql("drop table if exists test1_cached") sc.runSql("create table test1_cached as select * from test") @@ -318,92 +340,70 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // Caching Hive-partititioned tables + // Note: references to 'partition' for this section refer to a Hive-partition. ////////////////////////////////////////////////////////////////////////////// - test("Use regular CREATE TABLE and '_cached' suffix to create cached table") { - sc.runSql("drop table if exists empty_table_cached") - sc.runSql("create table empty_table_cached(key string, value string)") - assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached")) - assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached")) - sc.runSql("drop table if exists empty_table_cached") - } - - test("Use regular CREATE TABLE and table properties to create cached table") { - sc.runSql("drop table if exists empty_table_cached_tbl_props") - sc.runSql("""create table empty_table_cached_tbl_props(key string, value string) - TBLPROPERTIES('shark.cache' = 'true')""") - assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached_tbl_props")) - assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached_tbl_props")) - sc.runSql("drop table if exists empty_table_cached_tbl_props") - } - - test("Insert into empty cached table") { - assert(true) - } - test("Use regular CREATE TABLE and '_cached' suffix to create cached, partitioned table") { sc.runSql("drop table if exists empty_part_table_cached") sc.runSql("""create table empty_part_table_cached(key int, val string) partitioned by (keypart int)""") assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached")) assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached")) - sc.runSql("drop table if exists empty_part_table_cached") } test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { sc.runSql("drop table if exists empty_part_table_cached_tbl_props") sc.runSql("""create table empty_part_table_cached_tbl_props(key int, val string) - partitioned by (keypart int)""") + partitioned by (keypart int) TBLPROPERTIES('shark.cache' = 'true')""") assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached_tbl_props")) - sc.runSql("drop table if exists empty_part_table_cached_tbl_props") } - test("drop cached, partitioned table that has a single partition") { + test("alter cached table by adding a new partition") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("drop cached, partitioned table that has multiple partitions") { + test("alter cached table by dropping a partition") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("drop cached, partitioned table that has a UnionRDD partition") { + test("insert into a partition of a cached table") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("alter cached table by adding a new partition") { + test("insert overwrite a partition of a cached table") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("alter cached table by dropping a partition") { + test("scan cached, partitioned table that's empty") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("insert into a partition of a cached table") { + test("scan cached, partitioned table that has a single partition") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("insert overwrite a partition of a cached table") { + test("scan cached, partitioned table that has multiple partitions") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("scan cached, partitioned table that's empty") { + test("drop cached, partitioned table that has a single partition") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("scan cached, partitioned table that has a single partition") { + test("drop cached, partitioned table that has multiple partitions") { sc.runSql("drop table if exists srcpart_cached") assert(true) } - test("scan cached, partitioned table that has multiple partitions") { + test("drop cached, partitioned table that has a UnionRDD partition") { sc.runSql("drop table if exists srcpart_cached") assert(true) } @@ -434,7 +434,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // various data types ////////////////////////////////////////////////////////////////////////////// - test("various data types") { + test("boolean data type") { sc.sql("drop table if exists checkboolean") sc.sql("""create table checkboolean TBLPROPERTIES ("shark.cache" = "true") as select key, val, true as flag from test where key < "300" """) @@ -442,7 +442,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { select key, val, false as flag from test where key > "300" """) expectSql("select flag, count(*) from checkboolean group by flag order by flag asc", Array[String]("false\t208", "true\t292")) + } + test("byte data type") { sc.sql("drop table if exists checkbyte") sc.sql("drop table if exists checkbyte_cached") sc.sql("""create table checkbyte (key string, val string, flag tinyint) """) @@ -453,7 +455,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("""create table checkbyte_cached as select * from checkbyte""") expectSql("select flag, count(*) from checkbyte_cached group by flag order by flag asc", Array[String]("0\t208", "1\t292")) + } + test("binary data type") { sc.sql("drop table if exists checkbinary") sc.sql("drop table if exists checkbinary_cached") sc.sql("""create table checkbinary (key string, flag binary) """) @@ -464,7 +468,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("create table checkbinary_cached as select key, flag from checkbinary") expectSql("select cast(flag as string) as f from checkbinary_cached order by f asc limit 2", Array[String]("val_0", "val_0")) + } + test("short data type") { sc.sql("drop table if exists checkshort") sc.sql("drop table if exists checkshort_cached") sc.sql("""create table checkshort (key string, val string, flag smallint) """) From fe9f340886f6ab09ae37a5db5af738ccb813a448 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 20:26:06 -0700 Subject: [PATCH 063/331] Finish writing unit tests. --- .../memstore2/MemoryMetadataManager.scala | 10 +- src/test/scala/shark/SQLSuite.scala | 117 ++++++++++++++---- 2 files changed, 99 insertions(+), 28 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 9544b974..83d3cfd1 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -50,6 +50,12 @@ class MemoryMetadataManager { def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) + def containsHivePartition(key: String, partitionColumnValues: String) = { + val containsTable = _keyToMemoryTable.contains(key.toLowerCase) + return containsTable && + _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.contains(partitionColumnValues) + } + def add(key: String, isHivePartitioned: Boolean) { val memoryTable = new MemoryTable(key.toLowerCase, isHivePartitioned) if (isHivePartitioned) { @@ -66,7 +72,9 @@ class MemoryMetadataManager { } } - def getPartitionCacheMode(key: String, partitionColumnValues: String): CacheType.CacheType = { + def getHivePartitionCacheMode( + key: String, + partitionColumnValues: String): CacheType.CacheType = { _keyToMemoryTable.get(key.toLowerCase) match { case Some(memoryTable) => return memoryTable.keyToCacheModes(partitionColumnValues) case _ => return CacheType.NONE diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 84c6e64f..c8b8642e 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -359,53 +359,116 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("alter cached table by adding a new partition") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + sc.runSql("drop table if exists alter_part_cached") + sc.runSql("""create table alter_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_part_cached add partition(keypart = 1)""") + val tableName = alter_part_cached + val partitionColumn = "keypart=1" + assert(SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) + } + + // TODO(harvey): Create hadoop file for this. + test("alter cached table by adding a new partition, with a provided location") { + sc.runSql("drop table if exists alter_part_location_cached") + sc.runSql("""create table alter_part_location_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_part_location_cached add partition(keypart = 1)""") } test("alter cached table by dropping a partition") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + sc.runSql("drop table if exists alter_drop_part_cached") + sc.runSql("""create table alter_drop_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_drop_part_cached add partition(keypart = 1)""") + val tableName = alter_part_cached + val partitionColumn = "keypart=1" + assert(SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) + sc.runSql("""alter table alter_drop_part_cached drop partition(keypart = 1)""") + assert(!SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) } test("insert into a partition of a cached table") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + sc.runSql("drop table if exists insert_part_cached") + sc.runSql("""create table insert_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("insert into table insert_part_cached partition(keypart = 1) select * from test") + expectSql("select value from insert_part_cached where key = 407 and keypart = 1", "val_407") + } test("insert overwrite a partition of a cached table") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + sc.runSql("drop table if exists insert_over_part_cached") + sc.runSql("""create table insert_over_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("""insert into table insert_over_part_cached partition(keypart = 1) + select * from test""") + expectSql("""select value from insert_over_part_cached + where key = 407 and keypart = 1", "val_407""") + + sc.runSql("""insert overwrite table insert_over_part_cached partition(keypart = 1) + select value, -1 from test""") + expectSql("select value from insert_over_part_cached where key = 407 and keypart = 1", "-1") } test("scan cached, partitioned table that's empty") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + sc.runSql("drop table if exists empty_part_table_cached") + sc.runSql("""create table empty_part_table_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("select * from empty_part_table_cached", "") } test("scan cached, partitioned table that has a single partition") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + sc.runSql("drop table if exists scan_single_part_cached") + sc.runSql("""create table scan_single_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("insert into table scan_single_part_cached partition(keypart = 1) select * from test") + expectSql("select value from scan_single_part_cached where key = 407", "val_407") } test("scan cached, partitioned table that has multiple partitions") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) - } - - test("drop cached, partitioned table that has a single partition") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) - } - - test("drop cached, partitioned table that has multiple partitions") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + sc.runSql("drop table if exists scan_mult_part_cached") + sc.runSql("""create table scan_single_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("insert into table scan_mult_part_cached partition(keypart = 1) select * from test") + sc.runSql("insert into table scan_mult_part_cached partition(keypart = 5) select * from test") + sc.runSql("insert into table scan_mult_part_cached partition(keypart = 9) select * from test") + expectSql("select value, keypart from scan_mult_part_cached where key = 407 order by keypart", + Array("val_407\t1", "val_407\t5", "val_407\t9")) } - test("drop cached, partitioned table that has a UnionRDD partition") { - sc.runSql("drop table if exists srcpart_cached") - assert(true) + test("drop/unpersist cached, partitioned table that has multiple partitions") { + sc.runSql("drop table if exists drop_mult_part_cached") + sc.runSql("""create table drop_mult_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("insert into table drop_mult_part_cached partition(keypart = 1) select * from test") + sc.runSql("insert into table drop_mult_part_cached partition(keypart = 5) select * from test") + sc.runSql("insert into table drop_mult_part_cached partition(keypart = 9) select * from test") + val tableName = "drop_mult_part_cached" + val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") + val keypart5RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=5") + val keypart9RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=9") + sc.runSql("drop drop_mult_part_cached table ") + assert(!SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) + // All RDDs should have been unpersisted. + assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) + assert(keypart5RDD.getStorageLevel == StorageLevel.NONE) + assert(keypart9RDD.getStorageLevel == StorageLevel.NONE) + } + + test("drop cached partition represented by a UnionRDD (i.e., the result of multiple inserts)") { + sc.runSql("drop table if exists drop_union_part_cached") + sc.runSql("""create table drop_union_part_cached(key int, val string) + partitioned by (keypart int)""") + sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") + sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") + sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") + val tableName = "drop_union_part_cached" + val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") + sc.runSql("drop drop_union_part_cached table ") + assert(!SharkEnv.memoryMetadataManager.contains("drop_union_part_cached")) + // All RDDs should have been unpersisted. + assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) } ////////////////////////////////////////////////////////////////////////////// From dfb85567ff69d084564e812866126beb8dc159bb Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 20:48:24 -0700 Subject: [PATCH 064/331] Make CacheType a table property for Hive-partitioned tables. --- .../execution/MemoryStoreSinkOperator.scala | 8 ++-- .../scala/shark/execution/SparkDDLTask.scala | 5 +-- .../memstore2/MemoryMetadataManager.scala | 37 +++++++------------ .../scala/shark/memstore2/MemoryTable.scala | 6 +-- 4 files changed, 21 insertions(+), 35 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 6a616ee5..44066618 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -138,13 +138,13 @@ class MemoryStoreSinkOperator extends TerminalOperator { } if (!SharkEnv.memoryMetadataManager.contains(tableName)) { - SharkEnv.memoryMetadataManager.add(tableName, partitionColumnValues.isDefined) + // This is a CTAS. Add a new table entry to the Shark metadata. + SharkEnv.memoryMetadataManager.add(tableName, partitionColumnValues.isDefined, cacheMode) } if (SharkEnv.memoryMetadataManager.isHivePartitioned(tableName)) { - SharkEnv.memoryMetadataManager.putHivePartition( - tableName, partitionColumnValues.get, rdd, cacheMode) + SharkEnv.memoryMetadataManager.putHivePartition(tableName, partitionColumnValues.get, rdd) } else { - SharkEnv.memoryMetadataManager.put(tableName, rdd, cacheMode) + SharkEnv.memoryMetadataManager.put(tableName, rdd) } // Report remaining memory. diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index bad62730..7de5eb5e 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -64,10 +64,7 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab cacheMode: CacheType.CacheType) { val isHivePartitioned = (createTblDesc.getPartCols.size > 0) val tableName = createTblDesc.getTableName - SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned) - if (!isHivePartitioned) { - SharkEnv.memoryMetadataManager.put(tableName, new EmptyRDD(SharkEnv.sc), cacheMode) - } + SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned, cacheMode) } def addPartition( diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 83d3cfd1..be76a9dc 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -41,27 +41,13 @@ class MemoryMetadataManager { private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - def isHivePartitioned(key: String): Boolean = { - _keyToMemoryTable.get(key.toLowerCase) match { - case Some(memoryTable) => return memoryTable.isHivePartitioned - case None => return false - } - } - - def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) - - def containsHivePartition(key: String, partitionColumnValues: String) = { - val containsTable = _keyToMemoryTable.contains(key.toLowerCase) - return containsTable && - _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.contains(partitionColumnValues) - } - - def add(key: String, isHivePartitioned: Boolean) { + def add(key: String, isHivePartitioned: Boolean, cacheMode: CacheType.CacheType) { val memoryTable = new MemoryTable(key.toLowerCase, isHivePartitioned) if (isHivePartitioned) { memoryTable.keyToHivePartitions = new JavaHashMap[String, RDD[_]]() memoryTable.keyToCacheModes = new JavaHashMap[String, CacheType.CacheType]() } + memoryTable.cacheMode = cacheMode _keyToMemoryTable(key.toLowerCase) = memoryTable } @@ -72,19 +58,24 @@ class MemoryMetadataManager { } } - def getHivePartitionCacheMode( - key: String, - partitionColumnValues: String): CacheType.CacheType = { + def isHivePartitioned(key: String): Boolean = { _keyToMemoryTable.get(key.toLowerCase) match { - case Some(memoryTable) => return memoryTable.keyToCacheModes(partitionColumnValues) - case _ => return CacheType.NONE + case Some(memoryTable) => return memoryTable.isHivePartitioned + case None => return false } } - def put(key: String, rdd: RDD[_], cacheMode: CacheType.CacheType) { + def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) + + def containsHivePartition(key: String, partitionColumnValues: String) = { + val containsTable = _keyToMemoryTable.contains(key.toLowerCase) + return containsTable && + _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.contains(partitionColumnValues) + } + + def put(key: String, rdd: RDD[_]) { val memoryTable = _keyToMemoryTable(key.toLowerCase) memoryTable.tableRDD = rdd - memoryTable.cacheMode = cacheMode } def putHivePartition( diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 4f5e0480..23d26b09 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -44,12 +44,10 @@ class MemoryTable( // Should only be used if the table is not Hive-partitioned. var tableRDD: RDD[_] = _ - // CacheMode for the tableRDD. + // CacheMode for the table. + // This is common to all Hive-partitions (if applicable). var cacheMode: CacheType.CacheType = _ // Should only be used if a cached table is Hive-partitioned. var keyToHivePartitions: Map[String, RDD[_]] = _ - - // Map from Hive-Partition key to the cache mode for its RDD. - var keyToCacheModes: Map[String, CacheType.CacheType] = _ } From d38a04123db5a609405a2180a01f40be8565b01f Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 23 Sep 2013 22:55:29 -0700 Subject: [PATCH 065/331] Support for alter table add/drop cached Hive-partitions. --- .../scala/shark/execution/SparkDDLTask.scala | 21 +----- .../memstore2/MemoryMetadataManager.scala | 45 ++++++------ .../parse/SharkDDLSemanticAnalyzer.scala | 69 ++++++++++++++++--- 3 files changed, 85 insertions(+), 50 deletions(-) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 7de5eb5e..d8c0000c 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -41,15 +41,9 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab case creatTblDesc: CreateTableDesc => { createTable(hiveMetadataDb, creatTblDesc, work.cacheMode) } - case addPartitionDesc: AddPartitionDesc => { - addPartition(hiveMetadataDb, addPartitionDesc) - } - case dropTableDesc: DropTableDesc => { - dropTable(hiveMetadataDb, dropTableDesc) - } case _ => { throw new UnsupportedOperationException( - "Shark does not require specific DDL handling for: " + work.ddlDesc.getClass.getName) + "Shark does not require a DDL task for: " + work.ddlDesc.getClass.getName) } } @@ -65,18 +59,7 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab val isHivePartitioned = (createTblDesc.getPartCols.size > 0) val tableName = createTblDesc.getTableName SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned, cacheMode) - } - - def addPartition( - hiveMetadataDb: Hive, - addPartitionDesc: AddPartitionDesc) { - Unit - } - - def dropTable( - hiveMetadataDb: Hive, - dropTableDesc: DropTableDesc) { - Unit + SharkEnv.memoryMetadataManager.put(tableName, new EmptyRDD(SharkEnv.sc)) } override def getType = StageType.DDL diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index be76a9dc..964cc091 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -45,7 +45,6 @@ class MemoryMetadataManager { val memoryTable = new MemoryTable(key.toLowerCase, isHivePartitioned) if (isHivePartitioned) { memoryTable.keyToHivePartitions = new JavaHashMap[String, RDD[_]]() - memoryTable.keyToCacheModes = new JavaHashMap[String, CacheType.CacheType]() } memoryTable.cacheMode = cacheMode _keyToMemoryTable(key.toLowerCase) = memoryTable @@ -65,12 +64,12 @@ class MemoryMetadataManager { } } - def contains(key: String) = _keyToMemoryTable.contains(key.toLowerCase) + def contains(key: String): Boolean = _keyToMemoryTable.contains(key.toLowerCase) - def containsHivePartition(key: String, partitionColumnValues: String) = { + def containsHivePartition(key: String, partitionColumnValues: String): Boolean = { val containsTable = _keyToMemoryTable.contains(key.toLowerCase) - return containsTable && - _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.contains(partitionColumnValues) + return (containsTable && + _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.contains(partitionColumnValues)) } def put(key: String, rdd: RDD[_]) { @@ -81,10 +80,13 @@ class MemoryMetadataManager { def putHivePartition( key: String, partitionColumnValues: String, - rdd: RDD[_], - cacheMode: CacheType.CacheType) { + rdd: RDD[_]) { _keyToMemoryTable(key.toLowerCase).keyToHivePartitions(partitionColumnValues) = rdd - _keyToMemoryTable(key.toLowerCase).keyToCacheModes(partitionColumnValues) = cacheMode + } + + def dropHivePartition(key: String, partitionColumnValues: String) { + val rdd = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.remove(partitionColumnValues) + unpersistRDD(rdd.get) } def get(key: String): Option[RDD[_]] = { @@ -124,19 +126,6 @@ class MemoryMetadataManager { * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ def unpersist(key: String): Option[RDD[_]] = { - def unpersistRDD(rdd: RDD[_]): RDD[_] = { - rdd match { - case u: UnionRDD[_] => { - // Recursively unpersist() all RDDs that compose the UnionRDD. - u.unpersist() - u.rdds.map { - r => r.unpersist() - } - } - case r => r.unpersist() - } - return rdd - } def unpersistMemoryTable(memoryTable: MemoryTable): Option[RDD[_]] = { var unpersistedRDD: Option[RDD[_]] = None if (memoryTable.isHivePartitioned) { @@ -159,6 +148,20 @@ class MemoryMetadataManager { val memoryTableValue: Option[MemoryTable] = _keyToMemoryTable.remove(key.toLowerCase) return memoryTableValue.flatMap(unpersistMemoryTable(_)) } + + def unpersistRDD(rdd: RDD[_]): RDD[_] = { + rdd match { + case u: UnionRDD[_] => { + // Recursively unpersist() all RDDs that compose the UnionRDD. + u.unpersist() + u.rdds.map { + r => r.unpersist() + } + } + case r => r.unpersist() + } + return rdd + } } diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 7a16f443..f4ac65e6 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -1,37 +1,86 @@ package shark.parse +import java.util.{Map => JavaMap} + +import scala.collection.JavaConversions._ + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, HiveParser} +import org.apache.hadoop.hive.ql.plan.DDLWork import org.apache.spark.rdd.{UnionRDD, RDD} +import shark.execution.EmptyRDD import shark.{LogHelper, SharkEnv} class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) with LogHelper { - override def analyzeInternal(node: ASTNode): Unit = { - super.analyzeInternal(node) + override def analyzeInternal(ast: ASTNode): Unit = { + super.analyzeInternal(ast) - node.getToken.getType match { + ast.getToken.getType match { case HiveParser.TOK_DROPTABLE => { - SharkEnv.unpersist(getTableName(node)) + SharkEnv.unpersist(getTableName(ast)) } // Handle ALTER TABLE for cached, Hive-partitioned tables case HiveParser.TOK_ALTERTABLE_ADDPARTS => { - Unit + alterTableAddParts(ast) } case HiveParser.TOK_ALTERTABLE_DROPPARTS => { - Unit - } - case HiveParser.TOK_ALTERTABLE_PARTITION => { - Unit + alterTableDropParts(ast) } case _ => Unit } } + def alterTableAddParts(ast: ASTNode) { + val tableName = getTableName(ast) + val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val partitionColumns = table.getPartCols.map(_.getName) + if (SharkEnv.memoryMetadataManager.contains(tableName)) { + // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks + // and DDLWorks that contain AddPartitionDesc objects. + val addPartitionDescs = rootTasks.map(_.getWork.asInstanceOf[DDLWork].getAddPartitionDesc) + + for (addPartitionDesc <- addPartitionDescs) { + val partitionColumnToValue = addPartitionDesc.getPartSpec + val keyStr = makePartitionKeyStr(partitionColumns, partitionColumnToValue) + SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) + } + } + } + + def alterTableDropParts(ast: ASTNode) { + val tableName = getTableName(ast) + val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val partitionColumns = table.getPartCols.map(_.getName) + if (SharkEnv.memoryMetadataManager.contains(tableName)) { + // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with a DDLTask + // and a DDLWork that contains a DropTableDesc object. + val partSpecs = rootTasks.map( + _.getWork.asInstanceOf[DDLWork].getDropTblDesc).head.getPartSpecs + for (partSpec <- partSpecs) { + val partitionColumnToValue = partSpec.getPartSpecWithoutOperator + val keyStr = makePartitionKeyStr(partitionColumns, partitionColumnToValue) + SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) + } + } + } + private def getTableName(node: ASTNode): String = { BaseSemanticAnalyzer.getUnescapedName(node.getChild(0).asInstanceOf[ASTNode]) } -} \ No newline at end of file + + private def makePartitionKeyStr( + partitionColumns: Seq[String], + partitionColumnToValue: JavaMap[String, String]): String = { + // The keyStr is the string 'col1=value1/col2=value2'. + var keyStr = "" + for (partitionColumn <- partitionColumns) { + keyStr += "%s=%s/".format(partitionColumn, partitionColumnToValue(partitionColumn)) + } + keyStr = keyStr.dropRight(1) + return keyStr + } +} From ab7902694ba5ea3462b8e196bbfa04c35fb7b91c Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 24 Sep 2013 12:40:54 -0700 Subject: [PATCH 066/331] Initial support for INSERTs into cached, Hive-partitioned tables. Additions are mainly in SharkSemanticAnalyzer and operator tree initializatin. --- .../execution/MemoryStoreSinkOperator.scala | 12 ++++----- .../shark/execution/OperatorFactory.scala | 2 ++ .../memstore2/MemoryMetadataManager.scala | 14 +++++++++- .../parse/SharkDDLSemanticAnalyzer.scala | 21 ++++----------- .../shark/parse/SharkSemanticAnalyzer.scala | 26 ++++++++++++++++--- 5 files changed, 48 insertions(+), 27 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 44066618..4795f2dd 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -42,7 +42,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { @BeanProperty var shouldCompress: Boolean = _ @BeanProperty var storageLevel: StorageLevel = _ @BeanProperty var tableName: String = _ - @BeanProperty var partitionColumnValues: Option[String] = None // delimited by '/' + @BeanProperty var hivePartitionKey: String = _ @transient var cacheMode: CacheType.CacheType = _ @transient var useUnionRDD: Boolean = _ @transient var numColumns: Int = _ @@ -137,13 +137,13 @@ class MemoryStoreSinkOperator extends TerminalOperator { origRdd.context.runJob(origRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } - if (!SharkEnv.memoryMetadataManager.contains(tableName)) { - // This is a CTAS. Add a new table entry to the Shark metadata. - SharkEnv.memoryMetadataManager.add(tableName, partitionColumnValues.isDefined, cacheMode) - } if (SharkEnv.memoryMetadataManager.isHivePartitioned(tableName)) { - SharkEnv.memoryMetadataManager.putHivePartition(tableName, partitionColumnValues.get, rdd) + SharkEnv.memoryMetadataManager.putHivePartition(tableName, hivePartitionKey, rdd) } else { + if (!SharkEnv.memoryMetadataManager.contains(tableName)) { + // This is a CTAS. Add a new table entry to the Shark metadata. + SharkEnv.memoryMetadataManager.add(tableName, false /* isHivePartitioned */, cacheMode) + } SharkEnv.memoryMetadataManager.put(tableName, rdd) } diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 9fa8b4d7..1bd1ca55 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -48,6 +48,7 @@ object OperatorFactory extends LogHelper { tableName: String, storageLevel: StorageLevel, numColumns: Int, + hivePartitionKey: String, cacheMode: CacheType, useUnionRDD: Boolean): TerminalOperator = { val sinkOp = _newOperatorInstance( @@ -56,6 +57,7 @@ object OperatorFactory extends LogHelper { sinkOp.storageLevel = storageLevel sinkOp.numColumns = numColumns sinkOp.cacheMode = cacheMode + sinkOp.hivePartitionKey = hivePartitionKey sinkOp.useUnionRDD = useUnionRDD _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 964cc091..bfca2b0c 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -18,7 +18,7 @@ package shark.memstore2 import java.util.concurrent.ConcurrentHashMap -import java.util.{HashMap => JavaHashMap} +import java.util.{HashMap => JavaHashMap, Map => JavaMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ConcurrentMap @@ -167,6 +167,18 @@ class MemoryMetadataManager { object MemoryMetadataManager { + def makeHivePartitionKeyStr( + partitionColumns: Seq[String], + partitionColumnToValue: JavaMap[String, String]): String = { + // The keyStr is the string 'col1=value1/col2=value2'. + var keyStr = "" + for (partitionColumn <- partitionColumns) { + keyStr += "%s=%s/".format(partitionColumn, partitionColumnToValue(partitionColumn)) + } + keyStr = keyStr.dropRight(1) + return keyStr + } + /** Return a StorageLevel corresponding to its String name. */ def getStorageLevelFromString(s: String): StorageLevel = { if (s == null || s == "") { diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index f4ac65e6..381f29ff 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -1,7 +1,5 @@ package shark.parse -import java.util.{Map => JavaMap} - import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf @@ -12,6 +10,7 @@ import org.apache.spark.rdd.{UnionRDD, RDD} import shark.execution.EmptyRDD import shark.{LogHelper, SharkEnv} +import shark.memstore2.MemoryMetadataManager class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) with LogHelper { @@ -45,7 +44,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) for (addPartitionDesc <- addPartitionDescs) { val partitionColumnToValue = addPartitionDesc.getPartSpec - val keyStr = makePartitionKeyStr(partitionColumns, partitionColumnToValue) + val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( + partitionColumns, partitionColumnToValue) SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) } } @@ -62,7 +62,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) _.getWork.asInstanceOf[DDLWork].getDropTblDesc).head.getPartSpecs for (partSpec <- partSpecs) { val partitionColumnToValue = partSpec.getPartSpecWithoutOperator - val keyStr = makePartitionKeyStr(partitionColumns, partitionColumnToValue) + val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( + partitionColumns, partitionColumnToValue) SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) } } @@ -71,16 +72,4 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) private def getTableName(node: ASTNode): String = { BaseSemanticAnalyzer.getUnescapedName(node.getChild(0).asInstanceOf[ASTNode]) } - - private def makePartitionKeyStr( - partitionColumns: Seq[String], - partitionColumnToValue: JavaMap[String, String]): String = { - // The keyStr is the string 'col1=value1/col2=value2'. - var keyStr = "" - for (partitionColumn <- partitionColumns) { - keyStr += "%s=%s/".format(partitionColumn, partitionColumnToValue(partitionColumn)) - } - keyStr = keyStr.dropRight(1) - return keyStr - } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index fb7b7e79..2742ba87 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -171,11 +171,23 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) val storageLevel = RDDUtils.getStorageLevelOfCachedRDD(rdd) val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) + val table = db.getTable( + db.getCurrentDatabase(), tableName, false /* throwException */) + var hivePartitionKey = new String + if (table.isPartitioned) { + if (cacheMode == CacheType.TACHYON) { + throw new SemanticException( + "Shark does not support caching Hive-partitioned table(s) in Tachyon.") + } + hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey( + table.getPartCols.map(_.getName), qb) + } OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, cachedTableName, storageLevel, - _resSchema.size, // numColumns + _resSchema.size, /* numColumns */ + hivePartitionKey, cacheMode, useUnionRDD) } else { @@ -200,9 +212,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with hiveSinkOps.head, qb.getTableDesc.getTableName, storageLevel, - _resSchema.size, // numColumns + _resSchema.size, /* numColumns */ + new String, /* hivePartitionKey */ qb.getCacheModeForCreateTable, - false) + false /* useUnionRDD */) } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) } else { @@ -417,7 +430,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with object SharkSemanticAnalyzer extends LogHelper { - /** * The reflection object used to invoke convertRowSchemaToViewSchema. */ @@ -432,6 +444,12 @@ object SharkSemanticAnalyzer extends LogHelper { private val viewsExpandedField = classOf[SemanticAnalyzer].getDeclaredField("viewsExpanded") viewsExpandedField.setAccessible(true) + private def getHivePartitionKey(partitionColumns: Seq[String], qb: QB): String = { + val selectClauseKey = qb.getParseInfo.getClauseNamesForDest.first + val partitionColumnToValue = qb.getMetaData.getPartSpecForAlias(selectClauseKey) + return MemoryMetadataManager.makeHivePartitionKeyStr(partitionColumns, partitionColumnToValue) + } + /** * Given a Hive top operator (e.g. TableScanOperator), find all the file sink * operators (aka file output operator). From 1bf725d32efdce24784995f21b7c005ece3d26b9 Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 24 Sep 2013 12:46:15 -0700 Subject: [PATCH 067/331] Initial support for scanning cached, Hive-partitioned tables. --- .../scala/shark/execution/SparkTask.scala | 4 +- .../shark/execution/TableScanOperator.scala | 67 ++++++++++++++++--- 2 files changed, 59 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index 32241a47..5520b75a 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -116,6 +116,7 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { // topToTable maps Hive's TableScanOperator to the Table object. val topToTable: JHashMap[HiveTableScanOperator, Table] = work.pctx.getTopToTable() + val emptyPartnArray = new Array[Partition](0) // Add table metadata to TableScanOperators topOps.foreach { op => op.table = topToTable.get(op.hiveOp) @@ -127,7 +128,8 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { work.pctx.getOpToPartPruner().get(op.hiveOp), work.pctx.getConf(), "", work.pctx.getPrunedPartitions()) - op.parts = ppl.getConfirmedPartns.toArray ++ ppl.getUnknownPartns.toArray + op.parts = ppl.getConfirmedPartns.toArray(emptyPartnArray) ++ + ppl.getUnknownPartns.toArray(emptyPartnArray) val allParts = op.parts ++ ppl.getDeniedPartns.toArray if (allParts.size == 0) { op.firstConfPartDesc = new PartitionDesc(op.tableDesc, null) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index e82657e2..bbc4f82a 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -38,7 +38,7 @@ import shark.{SharkConfVars, SharkEnv, Utils} import shark.api.QueryExecutionException import shark.execution.optimization.ColumnPruner import shark.execution.serialization.{XmlSerializer, JavaSerializer} -import shark.memstore2.{CacheType, TablePartition, TablePartitionStats} +import shark.memstore2.{CacheType, MemoryMetadataManager, TablePartition, TablePartitionStats} import shark.tachyon.TachyonException @@ -51,7 +51,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // Metadata for Hive-partitions (i.e if the table was created from PARTITION BY). NULL if this // table isn't Hive-partitioned. Set in SparkTask::initializeTableScanTableDesc(). - @transient var parts: Array[Object] = _ + @transient var parts: Array[Partition] = _ // PartitionDescs are used during planning in Hive. This reference to a single PartitionDesc // is used to initialize partition ObjectInspectors. @@ -118,17 +118,23 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // 3. Hive table on HDFS (or other Hadoop storage) val cacheMode = CacheType.fromString( tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) - // TODO(harvey): Pruning Hive-partitioned, cached tables aren't supported yet. + // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). - val rdd = SharkEnv.memoryMetadataManager.get(tableKey).getOrElse { + if (!SharkEnv.memoryMetadataManager.contains(tableKey)) { logError("""|Table %s not found in block manager. |Are you trying to access a cached table from a Shark session other than |the one in which it was created?""".stripMargin.format(tableKey)) throw(new QueryExecutionException("Cached table not found")) } - logInfo("Loading table " + tableKey + " from Spark block manager") - createPrunedRdd(tableKey, rdd) + if (SharkEnv.memoryMetadataManager.isHivePartitioned(tableKey)) { + // Get a union of the Hive partition RDDs. + makeCachedPartitionRDD(tableKey, parts) + } else { + val rdd = SharkEnv.memoryMetadataManager.get(tableKey).get + logInfo("Loading table " + tableKey + " from Spark block manager") + createPrunedRdd(tableKey, rdd) + } } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { @@ -217,7 +223,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO override def preprocessRdd(rdd: RDD[_]): RDD[_] = { if (table.isPartitioned) { logInfo("Making %d Hive partitions".format(parts.size)) - makePartitionRDD(rdd) + makePartitionRDD(parts) } else { val tablePath = table.getPath.toString val ifc = table.getInputFormatClass @@ -239,18 +245,57 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } } + private def makeCachedPartitionRDD(tableKey: String, partitions: Array[Partition]): RDD[_] = { + val hivePartitionRDDSeq = partitions.map { partition => + val partDesc = Utilities.getPartitionDesc(partition) + // Get partition field info + val partSpec = partDesc.getPartSpec() + val partProps = partDesc.getProperties() + + val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + // Partitioning keys are delimited by "/" + val partColumns = partColsDelimited.trim().split("/").toSeq + // 'partValues[i]' contains the value for the partitioning column at 'partColumns[i]'. + // TODO(harvey): When is partSpec null? + val partValues = partColumns.map { key => + if (partSpec == null) { + new String + } else { + partSpec.get(key).toArray + } + }.toArray + val partKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partColumns, partSpec) + val hivePartitionRDD = SharkEnv.memoryMetadataManager.getHivePartition(tableKey, partKeyStr) + val serializedHconf = XmlSerializer.serialize(localHconf, localHconf) + + hivePartitionRDD.get.mapPartitions { iter => + val rowWithPartArr = new Array[Object](2) + // Map each tuple to a row object + iter.map { value => + rowWithPartArr.update(0, value.asInstanceOf[Object]) + rowWithPartArr.update(1, partValues) + rowWithPartArr.asInstanceOf[Object] + } + } + } + if (hivePartitionRDDSeq.size > 0) { + new UnionRDD(hivePartitionRDDSeq.head.context, hivePartitionRDDSeq) + } else { + new EmptyRDD(SharkEnv.sc) + } + } + + /** * Create an RDD for every partition column specified in the query. Note that for on-disk Hive * tables, a data directory is created for each partition corresponding to keys specified using * 'PARTITION BY'. */ - private def makePartitionRDD[T](rdd: RDD[T]): RDD[_] = { - val partitions = parts + private def makePartitionRDD(partitions: Array[Partition]): RDD[_] = { val rdds = new Array[RDD[Any]](partitions.size) var i = 0 - partitions.foreach { part => - val partition = part.asInstanceOf[Partition] + partitions.foreach { partition => val partDesc = Utilities.getPartitionDesc(partition) val tablePath = partition.getPartitionPath.toString From a168e618ed86e4171655b902bdbab15c9c2166da Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 24 Sep 2013 12:53:21 -0700 Subject: [PATCH 068/331] Fix SQLSuite test compilation issue ... --- src/test/scala/shark/SQLSuite.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index c8b8642e..f4891858 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -363,7 +363,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""create table alter_part_cached(key int, val string) partitioned by (keypart int)""") sc.runSql("""alter table alter_part_cached add partition(keypart = 1)""") - val tableName = alter_part_cached + val tableName = "alter_part_cached" val partitionColumn = "keypart=1" assert(SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) } @@ -381,7 +381,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""create table alter_drop_part_cached(key int, val string) partitioned by (keypart int)""") sc.runSql("""alter table alter_drop_part_cached add partition(keypart = 1)""") - val tableName = alter_part_cached + val tableName = "alter_drop_part_cached" val partitionColumn = "keypart=1" assert(SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) sc.runSql("""alter table alter_drop_part_cached drop partition(keypart = 1)""") @@ -404,7 +404,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""insert into table insert_over_part_cached partition(keypart = 1) select * from test""") expectSql("""select value from insert_over_part_cached - where key = 407 and keypart = 1", "val_407""") + where key = 407 and keypart = 1""", "val_407") sc.runSql("""insert overwrite table insert_over_part_cached partition(keypart = 1) select value, -1 from test""") @@ -415,7 +415,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists empty_part_table_cached") sc.runSql("""create table empty_part_table_cached(key int, val string) partitioned by (keypart int)""") - sc.runSql("select * from empty_part_table_cached", "") + expectSql("select * from empty_part_table_cached", "") } test("scan cached, partitioned table that has a single partition") { @@ -451,9 +451,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop drop_mult_part_cached table ") assert(!SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) // All RDDs should have been unpersisted. - assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) - assert(keypart5RDD.getStorageLevel == StorageLevel.NONE) - assert(keypart9RDD.getStorageLevel == StorageLevel.NONE) + //assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + //assert(keypart5RDD.get.getStorageLevel == StorageLevel.NONE) + //assert(keypart9RDD.get.getStorageLevel == StorageLevel.NONE) } test("drop cached partition represented by a UnionRDD (i.e., the result of multiple inserts)") { @@ -468,7 +468,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop drop_union_part_cached table ") assert(!SharkEnv.memoryMetadataManager.contains("drop_union_part_cached")) // All RDDs should have been unpersisted. - assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) + //assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) } ////////////////////////////////////////////////////////////////////////////// From bbf1c488685382d70a337e2c680413da8cd6225c Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 24 Sep 2013 14:31:59 -0700 Subject: [PATCH 069/331] Use SparkDDLTask for ALTER TABLE ADD/DROP PARTITION. Like CREATE TABLE handling, the SparkDDLTask is dependent on the Hive DDLTask. --- .../scala/shark/execution/SparkDDLTask.scala | 37 +++++++++++++++- .../memstore2/MemoryMetadataManager.scala | 9 ++-- .../parse/SharkDDLSemanticAnalyzer.scala | 42 +++++++++---------- .../shark/parse/SharkSemanticAnalyzer.scala | 8 ++-- src/test/scala/shark/SQLSuite.scala | 12 +++--- 5 files changed, 71 insertions(+), 37 deletions(-) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index d8c0000c..bdeeed96 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -17,6 +17,8 @@ package shark.execution +import scala.collection.JavaConversions._ + import org.apache.hadoop.hive.ql.{Context, DriverContext} import org.apache.hadoop.hive.ql.exec.{Task => HiveTask, TaskExecutionException} import org.apache.hadoop.hive.ql.metadata.Hive @@ -24,7 +26,7 @@ import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.plan.api.StageType import shark.{LogHelper, SharkEnv} -import shark.memstore2.CacheType +import shark.memstore2.{CacheType, MemoryMetadataManager} private[shark] class SparkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { @@ -41,6 +43,12 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab case creatTblDesc: CreateTableDesc => { createTable(hiveMetadataDb, creatTblDesc, work.cacheMode) } + case addPartitionDesc: AddPartitionDesc => { + addPartition(hiveMetadataDb, addPartitionDesc) + } + case dropTableDesc: DropTableDesc => { + dropTable(hiveMetadataDb, dropTableDesc) + } case _ => { throw new UnsupportedOperationException( "Shark does not require a DDL task for: " + work.ddlDesc.getClass.getName) @@ -62,6 +70,33 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab SharkEnv.memoryMetadataManager.put(tableName, new EmptyRDD(SharkEnv.sc)) } + def addPartition( + hiveMetadataDb: Hive, + addPartitionDesc: AddPartitionDesc) { + val tableName = addPartitionDesc.getTableName + val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val partitionColumns = table.getPartCols.map(_.getName) + val partitionColumnToValue = addPartitionDesc.getPartSpec + val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( + partitionColumns, partitionColumnToValue) + SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) + } + + def dropTable( + hiveMetadataDb: Hive, + dropTableDesc: DropTableDesc) { + val tableName = dropTableDesc.getTableName + val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val partitionColumns = table.getPartCols.map(_.getName) + val partSpecs = dropTableDesc.getPartSpecs + for (partSpec <- partSpecs) { + val partitionColumnToValue = partSpec.getPartSpecWithoutOperator + val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( + partitionColumns, partitionColumnToValue) + SharkEnv.memoryMetadataManager.dropHivePartition(tableName, keyStr) + } + } + override def getType = StageType.DDL override def getName = "DDL-SPARK" diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index bfca2b0c..cf88a571 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -81,11 +81,13 @@ class MemoryMetadataManager { key: String, partitionColumnValues: String, rdd: RDD[_]) { - _keyToMemoryTable(key.toLowerCase).keyToHivePartitions(partitionColumnValues) = rdd + val keyToHivePartitions = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions + keyToHivePartitions(partitionColumnValues) = rdd } def dropHivePartition(key: String, partitionColumnValues: String) { - val rdd = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.remove(partitionColumnValues) + val keyToHivePartitions = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions + val rdd = keyToHivePartitions.remove(partitionColumnValues) unpersistRDD(rdd.get) } @@ -97,7 +99,8 @@ class MemoryMetadataManager { } def getHivePartition(key: String, partitionColumnValues: String): Option[RDD[_]] = { - return _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.get(partitionColumnValues) + val keyToHivePartitions = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions + keyToHivePartitions.get(partitionColumnValues) } def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 381f29ff..a3c5a04b 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -3,12 +3,13 @@ package shark.parse import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.exec.TaskFactory import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, HiveParser} import org.apache.hadoop.hive.ql.plan.DDLWork import org.apache.spark.rdd.{UnionRDD, RDD} -import shark.execution.EmptyRDD +import shark.execution.{EmptyRDD, SparkDDLWork} import shark.{LogHelper, SharkEnv} import shark.memstore2.MemoryMetadataManager @@ -20,11 +21,14 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) ast.getToken.getType match { case HiveParser.TOK_DROPTABLE => { + // TODO(harvey): Execute this in SparkDDLTask. This somewhat works right now because + // unpersist() returns silently when the table doesn't exist. However, it + // ignores any drop protections. SharkEnv.unpersist(getTableName(ast)) } // Handle ALTER TABLE for cached, Hive-partitioned tables case HiveParser.TOK_ALTERTABLE_ADDPARTS => { - alterTableAddParts(ast) + analyzeAlterTableAddParts(ast) } case HiveParser.TOK_ALTERTABLE_DROPPARTS => { alterTableDropParts(ast) @@ -33,38 +37,30 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) } } - def alterTableAddParts(ast: ASTNode) { + def analyzeAlterTableAddParts(ast: ASTNode) { val tableName = getTableName(ast) - val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionColumns = table.getPartCols.map(_.getName) + // Create a SparkDDLTask only if the table is cached. if (SharkEnv.memoryMetadataManager.contains(tableName)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks // and DDLWorks that contain AddPartitionDesc objects. - val addPartitionDescs = rootTasks.map(_.getWork.asInstanceOf[DDLWork].getAddPartitionDesc) - - for (addPartitionDesc <- addPartitionDescs) { - val partitionColumnToValue = addPartitionDesc.getPartSpec - val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( - partitionColumns, partitionColumnToValue) - SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) + for (ddlTask <- rootTasks) { + val addPartitionDesc = ddlTask.getWork.asInstanceOf[DDLWork].getAddPartitionDesc + val sparkDDLWork = new SparkDDLWork(addPartitionDesc) + ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) } } } def alterTableDropParts(ast: ASTNode) { val tableName = getTableName(ast) - val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionColumns = table.getPartCols.map(_.getName) + // Create a SparkDDLTask only if the table is cached. if (SharkEnv.memoryMetadataManager.contains(tableName)) { - // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with a DDLTask - // and a DDLWork that contains a DropTableDesc object. - val partSpecs = rootTasks.map( - _.getWork.asInstanceOf[DDLWork].getDropTblDesc).head.getPartSpecs - for (partSpec <- partSpecs) { - val partitionColumnToValue = partSpec.getPartSpecWithoutOperator - val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( - partitionColumns, partitionColumnToValue) - SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) + // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks + // and DDLWorks that contain AddPartitionDesc objects. + for (ddlTask <- rootTasks) { + val dropTableDesc = ddlTask.getWork.asInstanceOf[DDLWork].getDropTblDesc + val sparkDDLWork = new SparkDDLWork(dropTableDesc) + ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) } } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 2742ba87..a5815cd9 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -172,7 +172,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val storageLevel = RDDUtils.getStorageLevelOfCachedRDD(rdd) val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) val table = db.getTable( - db.getCurrentDatabase(), tableName, false /* throwException */) + db.getCurrentDatabase(), cachedTableName, false /* throwException */) var hivePartitionKey = new String if (table.isPartitioned) { if (cacheMode == CacheType.TACHYON) { @@ -416,9 +416,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // CREATE TABLE is valid. So, hook a SharkDDLTask as a dependent of the Hive DDLTask so that // Shark metadata is updated only if the Hive task execution is successful. val hiveDDLTask = ddlTasks.head; - val sharkDDLWork = new SparkDDLWork(createTableDesc) - sharkDDLWork.cacheMode = cacheMode - hiveDDLTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) + val sparkDDLWork = new SparkDDLWork(createTableDesc) + sparkDDLWork.cacheMode = cacheMode + hiveDDLTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) } queryBlock.setCacheModeForCreateTable(cacheMode) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index f4891858..be40ec35 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -369,12 +369,12 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } // TODO(harvey): Create hadoop file for this. - test("alter cached table by adding a new partition, with a provided location") { - sc.runSql("drop table if exists alter_part_location_cached") - sc.runSql("""create table alter_part_location_cached(key int, val string) - partitioned by (keypart int)""") - sc.runSql("""alter table alter_part_location_cached add partition(keypart = 1)""") - } + // test("alter cached table by adding a new partition, with a provided location") { + // sc.runSql("drop table if exists alter_part_location_cached") + // sc.runSql("""create table alter_part_location_cached(key int, val string) + // partitioned by (keypart int)""") + // sc.runSql("""alter table alter_part_location_cached add partition(keypart = 1)""") + // } test("alter cached table by dropping a partition") { sc.runSql("drop table if exists alter_drop_part_cached") From 7d73c57917677fbcf5ffd2b6a17b898aa152c73c Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 24 Sep 2013 16:06:36 -0700 Subject: [PATCH 070/331] Add error checking in MemoryTable, motivated by a bug fix in MemoryStoreSinkOperator. --- .../execution/MemoryStoreSinkOperator.scala | 22 ++++++++---- .../scala/shark/memstore2/MemoryTable.scala | 36 +++++++++++++++++-- .../shark/parse/SharkSemanticAnalyzer.scala | 15 ++++---- src/test/scala/shark/SQLSuite.scala | 28 +++++++-------- 4 files changed, 70 insertions(+), 31 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 4795f2dd..1bb2a5e5 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -97,6 +97,8 @@ class MemoryStoreSinkOperator extends TerminalOperator { } } + val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned(tableName) + if (tachyonWriter != null) { // Put the table in Tachyon. op.logInfo("Putting RDD for %s in Tachyon".format(tableName)) @@ -126,25 +128,33 @@ class MemoryStoreSinkOperator extends TerminalOperator { val origRdd = rdd if (useUnionRDD) { + val oldRdd: Option[RDD[_]] = + if (isHivePartitioned) { + SharkEnv.memoryMetadataManager.getHivePartition(tableName, hivePartitionKey) + } else { + SharkEnv.memoryMetadataManager.get(tableName) + } // If this is an insert, find the existing RDD and create a union of the two, and then // put the union into the meta data tracker. - rdd = rdd.union( - SharkEnv.memoryMetadataManager.get(tableName).get.asInstanceOf[RDD[TablePartition]]) + rdd = oldRdd match { + case Some(definedRdd) => rdd.union(oldRdd.get.asInstanceOf[RDD[TablePartition]]) + // The oldRdd can be missing if this is an INSERT into a new Hive-partition. + case None => rdd + } } - rdd.setName(tableName) - // Run a job on the original RDD to force it to go into cache. origRdd.context.runJob(origRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } - - if (SharkEnv.memoryMetadataManager.isHivePartitioned(tableName)) { + if (isHivePartitioned) { SharkEnv.memoryMetadataManager.putHivePartition(tableName, hivePartitionKey, rdd) + rdd.setName(tableName + "(" + hivePartitionKey + ")") } else { if (!SharkEnv.memoryMetadataManager.contains(tableName)) { // This is a CTAS. Add a new table entry to the Shark metadata. SharkEnv.memoryMetadataManager.add(tableName, false /* isHivePartitioned */, cacheMode) } SharkEnv.memoryMetadataManager.put(tableName, rdd) + rdd.setName(tableName) } // Report remaining memory. diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 23d26b09..bd7081a8 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -42,12 +42,42 @@ class MemoryTable( val isHivePartitioned: Boolean) { // Should only be used if the table is not Hive-partitioned. - var tableRDD: RDD[_] = _ + private var _tableRDD: RDD[_] = _ + + // Should only be used if a cached table is Hive-partitioned. + private var _keyToHivePartitions: Map[String, RDD[_]] = _ // CacheMode for the table. // This is common to all Hive-partitions (if applicable). var cacheMode: CacheType.CacheType = _ - // Should only be used if a cached table is Hive-partitioned. - var keyToHivePartitions: Map[String, RDD[_]] = _ + def tableRDD: RDD[_] = { + assert ( + !isHivePartitioned, + "Table " + tableName + " is Hive-partitioned. Use MemoryTableDesc::hivePartitionRDDs() " + + "to get RDDs corresponding to partition columns" + ) + return _tableRDD + } + + def tableRDD_= (value: RDD[_]) { + assert( + !isHivePartitioned, + "Table " + tableName + " is Hive-partitioned. Pass in a map of pairs " + + "to the 'keyToHivePartitions =' setter." + ) + _tableRDD = value + } + + def keyToHivePartitions: Map[String, RDD[_]] = { + assert(isHivePartitioned, + "Table " + tableName + " is not Hive-partitioned. Use tableRDD() to get its RDD.") + _keyToHivePartitions + } + + def keyToHivePartitions_= (value: Map[String, RDD[_]]) { + assert(isHivePartitioned, + "Table " + tableName + " is not Hive-partitioned. Use 'tableRDD =' to set the RDD.") + _keyToHivePartitions = value + } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index a5815cd9..3f0ea848 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -171,16 +171,13 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) val storageLevel = RDDUtils.getStorageLevelOfCachedRDD(rdd) val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) - val table = db.getTable( - db.getCurrentDatabase(), cachedTableName, false /* throwException */) var hivePartitionKey = new String - if (table.isPartitioned) { + if (SharkEnv.memoryMetadataManager.isHivePartitioned(cachedTableName)) { if (cacheMode == CacheType.TACHYON) { throw new SemanticException( "Shark does not support caching Hive-partitioned table(s) in Tachyon.") } - hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey( - table.getPartCols.map(_.getName), qb) + hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey(qb) } OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, @@ -444,9 +441,11 @@ object SharkSemanticAnalyzer extends LogHelper { private val viewsExpandedField = classOf[SemanticAnalyzer].getDeclaredField("viewsExpanded") viewsExpandedField.setAccessible(true) - private def getHivePartitionKey(partitionColumns: Seq[String], qb: QB): String = { - val selectClauseKey = qb.getParseInfo.getClauseNamesForDest.first - val partitionColumnToValue = qb.getMetaData.getPartSpecForAlias(selectClauseKey) + private def getHivePartitionKey(qb: QB): String = { + val selectClauseKey = qb.getParseInfo.getClauseNamesForDest.head + val destPartition = qb.getMetaData.getDestPartitionForAlias(selectClauseKey) + val partitionColumns = destPartition.getTable.getPartCols.map(_.getName) + val partitionColumnToValue = destPartition.getSpec return MemoryMetadataManager.makeHivePartitionKeyStr(partitionColumns, partitionColumnToValue) } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index be40ec35..a2007b4e 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -344,7 +344,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// test("Use regular CREATE TABLE and '_cached' suffix to create cached, partitioned table") { sc.runSql("drop table if exists empty_part_table_cached") - sc.runSql("""create table empty_part_table_cached(key int, val string) + sc.runSql("""create table empty_part_table_cached(key int, value string) partitioned by (keypart int)""") assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached")) assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached")) @@ -352,7 +352,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { sc.runSql("drop table if exists empty_part_table_cached_tbl_props") - sc.runSql("""create table empty_part_table_cached_tbl_props(key int, val string) + sc.runSql("""create table empty_part_table_cached_tbl_props(key int, value string) partitioned by (keypart int) TBLPROPERTIES('shark.cache' = 'true')""") assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached_tbl_props")) @@ -360,7 +360,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("alter cached table by adding a new partition") { sc.runSql("drop table if exists alter_part_cached") - sc.runSql("""create table alter_part_cached(key int, val string) + sc.runSql("""create table alter_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("""alter table alter_part_cached add partition(keypart = 1)""") val tableName = "alter_part_cached" @@ -378,7 +378,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("alter cached table by dropping a partition") { sc.runSql("drop table if exists alter_drop_part_cached") - sc.runSql("""create table alter_drop_part_cached(key int, val string) + sc.runSql("""create table alter_drop_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("""alter table alter_drop_part_cached add partition(keypart = 1)""") val tableName = "alter_drop_part_cached" @@ -390,7 +390,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("insert into a partition of a cached table") { sc.runSql("drop table if exists insert_part_cached") - sc.runSql("""create table insert_part_cached(key int, val string) + sc.runSql("""create table insert_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("insert into table insert_part_cached partition(keypart = 1) select * from test") expectSql("select value from insert_part_cached where key = 407 and keypart = 1", "val_407") @@ -399,7 +399,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("insert overwrite a partition of a cached table") { sc.runSql("drop table if exists insert_over_part_cached") - sc.runSql("""create table insert_over_part_cached(key int, val string) + sc.runSql("""create table insert_over_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("""insert into table insert_over_part_cached partition(keypart = 1) select * from test""") @@ -413,14 +413,14 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("scan cached, partitioned table that's empty") { sc.runSql("drop table if exists empty_part_table_cached") - sc.runSql("""create table empty_part_table_cached(key int, val string) + sc.runSql("""create table empty_part_table_cached(key int, value string) partitioned by (keypart int)""") - expectSql("select * from empty_part_table_cached", "") + expectSql("select count(*) from empty_part_table_cached", "0") } test("scan cached, partitioned table that has a single partition") { sc.runSql("drop table if exists scan_single_part_cached") - sc.runSql("""create table scan_single_part_cached(key int, val string) + sc.runSql("""create table scan_single_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("insert into table scan_single_part_cached partition(keypart = 1) select * from test") expectSql("select value from scan_single_part_cached where key = 407", "val_407") @@ -428,7 +428,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("scan cached, partitioned table that has multiple partitions") { sc.runSql("drop table if exists scan_mult_part_cached") - sc.runSql("""create table scan_single_part_cached(key int, val string) + sc.runSql("""create table scan_single_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("insert into table scan_mult_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table scan_mult_part_cached partition(keypart = 5) select * from test") @@ -439,7 +439,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("drop/unpersist cached, partitioned table that has multiple partitions") { sc.runSql("drop table if exists drop_mult_part_cached") - sc.runSql("""create table drop_mult_part_cached(key int, val string) + sc.runSql("""create table drop_mult_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("insert into table drop_mult_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table drop_mult_part_cached partition(keypart = 5) select * from test") @@ -448,7 +448,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") val keypart5RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=5") val keypart9RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=9") - sc.runSql("drop drop_mult_part_cached table ") + sc.runSql("drop table drop_mult_part_cached table ") assert(!SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) // All RDDs should have been unpersisted. //assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) @@ -458,14 +458,14 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("drop cached partition represented by a UnionRDD (i.e., the result of multiple inserts)") { sc.runSql("drop table if exists drop_union_part_cached") - sc.runSql("""create table drop_union_part_cached(key int, val string) + sc.runSql("""create table drop_union_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") val tableName = "drop_union_part_cached" val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") - sc.runSql("drop drop_union_part_cached table ") + sc.runSql("drop table drop_union_part_cached table ") assert(!SharkEnv.memoryMetadataManager.contains("drop_union_part_cached")) // All RDDs should have been unpersisted. //assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) From a53d195f344ddeaa71388eb980cb64f1fda157ae Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 24 Sep 2013 16:26:32 -0700 Subject: [PATCH 071/331] =?UTF-8?q?Add=20some=20RDD=20storage=20level=20ut?= =?UTF-8?q?ilities,=20squash=20a=20bug=20in=20SharkSemanticAnalyzer=20that?= =?UTF-8?q?=20was=20fetching=20RDDs=20the=20wrong=20way=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- src/main/scala/shark/execution/RDDUtils.scala | 20 +++++--- .../memstore2/MemoryMetadataManager.scala | 10 ++++ .../shark/parse/SharkSemanticAnalyzer.scala | 51 +++++++++---------- 3 files changed, 47 insertions(+), 34 deletions(-) diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 2c52840f..60ac9eee 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -36,16 +36,20 @@ object RDDUtils { def getStorageLevelOfCachedRDD(rdd: RDD[_]): StorageLevel = { rdd match { - case u: UnionRDD[_] => u.rdds.foldLeft(rdd.getStorageLevel) { - (s, r) => { - if (s == StorageLevel.NONE) { - getStorageLevelOfCachedRDD(r) - } else { - s - } + case u: UnionRDD[_] => getStorageLevelOfCachedRDDs(u.rdds) + case _ => rdd.getStorageLevel + } + } + + def getStorageLevelOfCachedRDDs(rdds: Seq[RDD[_]]): StorageLevel = { + rdds.foldLeft(StorageLevel.NONE) { + (s, r) => { + if (s == StorageLevel.NONE) { + getStorageLevelOfCachedRDD(r) + } else { + s } } - case _ => rdd.getStorageLevel } } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index cf88a571..88bd2e29 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -26,6 +26,7 @@ import scala.collection.mutable.ConcurrentMap import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.storage.StorageLevel +import shark.execution.RDDUtils import shark.SharkConfVars import shark.SharkEnv @@ -118,6 +119,15 @@ class MemoryMetadataManager { _keyToMemoryTable.keys.collect { case k: String => k } toSeq } + def getStorageLevel(key: String): StorageLevel = { + if (isHivePartitioned(key)) { + val hivePartitionRDDs = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.values + return RDDUtils.getStorageLevelOfCachedRDDs(hivePartitionRDDs.toSeq) + } else { + return RDDUtils.getStorageLevelOfCachedRDD(get(key.toLowerCase).get) + } + } + /** * Used to drop a table from the Spark in-memory cache and/or disk. All metadata * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) tracked by Shark is deleted diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 3f0ea848..522476ed 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -164,35 +164,34 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } else { // Otherwise, check if we are inserting into a table that was cached. val cachedTableName = tableName.split('.')(1) // Ignore the database name - SharkEnv.memoryMetadataManager.get(cachedTableName) match { - case Some(rdd) => { - if (hiveSinkOps.size == 1) { - // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. - val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) - val storageLevel = RDDUtils.getStorageLevelOfCachedRDD(rdd) - val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) - var hivePartitionKey = new String - if (SharkEnv.memoryMetadataManager.isHivePartitioned(cachedTableName)) { - if (cacheMode == CacheType.TACHYON) { - throw new SemanticException( - "Shark does not support caching Hive-partitioned table(s) in Tachyon.") - } - hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey(qb) + if (SharkEnv.memoryMetadataManager.contains(cachedTableName)) { + if (hiveSinkOps.size == 1) { + // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. + val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) + val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) + var hivePartitionKey = new String + if (SharkEnv.memoryMetadataManager.isHivePartitioned(cachedTableName)) { + if (cacheMode == CacheType.TACHYON) { + throw new SemanticException( + "Shark does not support caching Hive-partitioned table(s) in Tachyon.") } - OperatorFactory.createSharkMemoryStoreOutputPlan( - hiveSinkOp, - cachedTableName, - storageLevel, - _resSchema.size, /* numColumns */ - hivePartitionKey, - cacheMode, - useUnionRDD) - } else { - throw new SemanticException( - "Shark does not support updating cached table(s) with multiple INSERTs") + hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey(qb) } + val storageLevel = SharkEnv.memoryMetadataManager.getStorageLevel(cachedTableName) + OperatorFactory.createSharkMemoryStoreOutputPlan( + hiveSinkOp, + cachedTableName, + storageLevel, + _resSchema.size, /* numColumns */ + hivePartitionKey, + cacheMode, + useUnionRDD) + } else { + throw new SemanticException( + "Shark does not support updating cached table(s) with multiple INSERTs") } - case None => OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) + } else { + OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) } } } From 7e14232be189d5a98a0c18efd4378e2f8d9072d2 Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 24 Sep 2013 17:04:35 -0700 Subject: [PATCH 072/331] Insert into cached partition works! --- .../scala/shark/execution/SparkDDLTask.scala | 4 +++- .../shark/execution/TableScanOperator.scala | 18 +++++++++++------- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index bdeeed96..5e156be7 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -67,7 +67,9 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab val isHivePartitioned = (createTblDesc.getPartCols.size > 0) val tableName = createTblDesc.getTableName SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned, cacheMode) - SharkEnv.memoryMetadataManager.put(tableName, new EmptyRDD(SharkEnv.sc)) + if (!isHivePartitioned) { + SharkEnv.memoryMetadataManager.put(tableName, new EmptyRDD(SharkEnv.sc)) + } } def addPartition( diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index bbc4f82a..7552db6f 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -269,12 +269,17 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO val serializedHconf = XmlSerializer.serialize(localHconf, localHconf) hivePartitionRDD.get.mapPartitions { iter => - val rowWithPartArr = new Array[Object](2) - // Map each tuple to a row object - iter.map { value => - rowWithPartArr.update(0, value.asInstanceOf[Object]) - rowWithPartArr.update(1, partValues) - rowWithPartArr.asInstanceOf[Object] + if (iter.hasNext) { + // Map each tuple to a row object + val rowWithPartArr = new Array[Object](2) + val tablePartition = iter.next.asInstanceOf[TablePartition] + tablePartition.iterator.map { value => + rowWithPartArr.update(0, value.asInstanceOf[Object]) + rowWithPartArr.update(1, partValues) + rowWithPartArr.asInstanceOf[Object] + } + } else { + Iterator() } } } @@ -285,7 +290,6 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } } - /** * Create an RDD for every partition column specified in the query. Note that for on-disk Hive * tables, a data directory is created for each partition corresponding to keys specified using From 7211b462250b6a334a868f23e5c652128c9554f6 Mon Sep 17 00:00:00 2001 From: Harvey Date: Wed, 25 Sep 2013 00:44:10 -0700 Subject: [PATCH 073/331] ALTER/INSERT/DROP on cached, Hive partitions work! --- .../shark/execution/TableScanOperator.scala | 9 +++++---- .../shark/memstore2/MemoryMetadataManager.scala | 2 +- src/test/scala/shark/SQLSuite.scala | 17 ++++++++--------- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 7552db6f..2c7fdd44 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -128,7 +128,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO throw(new QueryExecutionException("Cached table not found")) } if (SharkEnv.memoryMetadataManager.isHivePartitioned(tableKey)) { - // Get a union of the Hive partition RDDs. + // Get the union of RDDs repesenting the selected Hive partition(s). makeCachedPartitionRDD(tableKey, parts) } else { val rdd = SharkEnv.memoryMetadataManager.get(tableKey).get @@ -221,8 +221,9 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO * Create a RDD representing the table (with or without partitions). */ override def preprocessRdd(rdd: RDD[_]): RDD[_] = { + val tableKey: String = tableDesc.getTableName.split('.')(1) if (table.isPartitioned) { - logInfo("Making %d Hive partitions".format(parts.size)) + logInfo("Making %d Hive partitions for table %s".format(parts.size, tableKey)) makePartitionRDD(parts) } else { val tablePath = table.getPath.toString @@ -261,7 +262,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO if (partSpec == null) { new String } else { - partSpec.get(key).toArray + new String(partSpec.get(key)) } }.toArray val partKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partColumns, partSpec) @@ -286,7 +287,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO if (hivePartitionRDDSeq.size > 0) { new UnionRDD(hivePartitionRDDSeq.head.context, hivePartitionRDDSeq) } else { - new EmptyRDD(SharkEnv.sc) + SharkEnv.sc.makeRDD(Seq[Object]()) } } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 88bd2e29..e9b2a58d 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -120,7 +120,7 @@ class MemoryMetadataManager { } def getStorageLevel(key: String): StorageLevel = { - if (isHivePartitioned(key)) { + if (isHivePartitioned(key.toLowerCase)) { val hivePartitionRDDs = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.values return RDDUtils.getStorageLevelOfCachedRDDs(hivePartitionRDDs.toSeq) } else { diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index a2007b4e..29ebe256 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -405,9 +405,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { select * from test""") expectSql("""select value from insert_over_part_cached where key = 407 and keypart = 1""", "val_407") - sc.runSql("""insert overwrite table insert_over_part_cached partition(keypart = 1) - select value, -1 from test""") + select key, -1 from test""") expectSql("select value from insert_over_part_cached where key = 407 and keypart = 1", "-1") } @@ -423,18 +422,18 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""create table scan_single_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("insert into table scan_single_part_cached partition(keypart = 1) select * from test") - expectSql("select value from scan_single_part_cached where key = 407", "val_407") + expectSql("select * from scan_single_part_cached where key = 407", "407\tval_407\t1") } test("scan cached, partitioned table that has multiple partitions") { sc.runSql("drop table if exists scan_mult_part_cached") - sc.runSql("""create table scan_single_part_cached(key int, value string) + sc.runSql("""create table scan_mult_part_cached(key int, value string) partitioned by (keypart int)""") sc.runSql("insert into table scan_mult_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table scan_mult_part_cached partition(keypart = 5) select * from test") sc.runSql("insert into table scan_mult_part_cached partition(keypart = 9) select * from test") - expectSql("select value, keypart from scan_mult_part_cached where key = 407 order by keypart", - Array("val_407\t1", "val_407\t5", "val_407\t9")) + expectSql("select * from scan_mult_part_cached where key = 407 order by keypart", + Array("407\tval_407\t1", "407\tval_407\t5", "407\tval_407\t9")) } test("drop/unpersist cached, partitioned table that has multiple partitions") { @@ -448,8 +447,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") val keypart5RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=5") val keypart9RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=9") - sc.runSql("drop table drop_mult_part_cached table ") - assert(!SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) + sc.runSql("drop table drop_mult_part_cached ") + assert(!SharkEnv.memoryMetadataManager.contains("drop_mult_part_cached")) // All RDDs should have been unpersisted. //assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) //assert(keypart5RDD.get.getStorageLevel == StorageLevel.NONE) @@ -465,7 +464,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") val tableName = "drop_union_part_cached" val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") - sc.runSql("drop table drop_union_part_cached table ") + sc.runSql("drop table drop_union_part_cached") assert(!SharkEnv.memoryMetadataManager.contains("drop_union_part_cached")) // All RDDs should have been unpersisted. //assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) From c7ff41d0d09097de028272a6d8d549a01c72e8c4 Mon Sep 17 00:00:00 2001 From: harshars Date: Wed, 11 Sep 2013 10:10:33 -0700 Subject: [PATCH 074/331] Bug fix in Pruning between clauses. When the second evaluator is a function, we should not throw an exception Conflicts: src/test/scala/shark/SQLSuite.scala --- .../shark/execution/MapSplitPruning.scala | 23 ++++++++++++------- src/test/scala/shark/SQLSuite.scala | 23 ++++++++++++++++--- 2 files changed, 35 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index 5959660b..f7b1634b 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -57,11 +57,18 @@ object MapSplitPruning { e.genericUDF match { case _: GenericUDFOPAnd => test(s, e.children(0)) && test(s, e.children(1)) case _: GenericUDFOPOr => test(s, e.children(0)) || test(s, e.children(1)) - case _: GenericUDFBetween => - testBetweenPredicate(s, e.children(0).asInstanceOf[ExprNodeConstantEvaluator], - e.children(1).asInstanceOf[ExprNodeColumnEvaluator], - e.children(2).asInstanceOf[ExprNodeConstantEvaluator], - e.children(3).asInstanceOf[ExprNodeConstantEvaluator]) + case _: GenericUDFBetween => + val col = e.children(1) + if (col.isInstanceOf[ExprNodeColumnEvaluator]) { + testBetweenPredicate(s, e.children(0).asInstanceOf[ExprNodeConstantEvaluator], + col.asInstanceOf[ExprNodeColumnEvaluator], + e.children(2).asInstanceOf[ExprNodeConstantEvaluator], + e.children(3).asInstanceOf[ExprNodeConstantEvaluator]) + } else { + //cannot prune function based evaluators in general. + true + } + case _: GenericUDFIn => testInPredicate(s, e.children(0).asInstanceOf[ExprNodeColumnEvaluator], e.children.drop(1)) case udf: GenericUDFBaseCompare => @@ -93,20 +100,20 @@ object MapSplitPruning { true } } - + def testBetweenPredicate( s: TablePartitionStats, invertEval: ExprNodeConstantEvaluator, columnEval: ExprNodeColumnEvaluator, leftEval: ExprNodeConstantEvaluator, rightEval: ExprNodeConstantEvaluator): Boolean = { - + val field = columnEval.field.asInstanceOf[IDStructField] val columnStats = s.stats(field.fieldID) val leftValue: Object = leftEval.expr.getValue val rightValue: Object = rightEval.expr.getValue val invertValue: Boolean = invertEval.expr.getValue.asInstanceOf[Boolean] - + if (columnStats != null) { val exists = (columnStats :>< (leftValue , rightValue)) if (invertValue) !exists else exists diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 1d2190c4..c22b9f64 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -349,7 +349,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // various data types ////////////////////////////////////////////////////////////////////////////// - + test("various data types") { sc.sql("drop table if exists checkboolean") sc.sql("""create table checkboolean TBLPROPERTIES ("shark.cache" = "true") as @@ -369,7 +369,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("""create table checkbyte_cached as select * from checkbyte""") expectSql("select flag, count(*) from checkbyte_cached group by flag order by flag asc", Array[String]("0\t208", "1\t292")) - + sc.sql("drop table if exists checkbinary") sc.sql("drop table if exists checkbinary_cached") sc.sql("""create table checkbinary (key string, flag binary) """) @@ -380,7 +380,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("create table checkbinary_cached as select key, flag from checkbinary") expectSql("select cast(flag as string) as f from checkbinary_cached order by f asc limit 2", Array[String]("val_0", "val_0")) - + sc.sql("drop table if exists checkshort") sc.sql("drop table if exists checkshort_cached") sc.sql("""create table checkshort (key string, val string, flag smallint) """) @@ -393,6 +393,23 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { Array[String]("23\t292", "36\t208")) } + ////////////////////////////////////////////////////////////////////////////// + // Test function based pruning Bug + ////////////////////////////////////////////////////////////////////////////// + + test("map pruning with functions in between clause") { + sc.sql("drop table if exists mapsplitfunc") + sc.sql("drop table if exists mapsplitfunc_cached") + sc.sql("create table mapsplitfunc(k bigint, v string)") + sc.sql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + OVERWRITE INTO TABLE mapsplitfunc""") + sc.sql("create table mapsplitfunc_cached as select * from mapsplitfunc") + expectSql("""select count(*) from mapsplitfunc_cached + where month(from_unixtime(k)) between "1" and "12" """, Array[String]("500")) + expectSql("""select count(*) from mapsplitfunc_cached + where year(from_unixtime(k)) between "2013" and "2014" """, Array[String]("0")) + } + ////////////////////////////////////////////////////////////////////////////// // SharkContext APIs (e.g. sql2rdd, sql) ////////////////////////////////////////////////////////////////////////////// From 54be94c60de98e3e610bb20046cb4c351542ae85 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 Sep 2013 16:53:36 -0700 Subject: [PATCH 075/331] Moved the test for partition pruning into the appropriate block. --- src/test/scala/shark/SQLSuite.scala | 32 +++++++++++++---------------- 1 file changed, 14 insertions(+), 18 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index c22b9f64..747171b6 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -336,7 +336,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } ////////////////////////////////////////////////////////////////////////////// - // Sel Star + // Partition pruning ////////////////////////////////////////////////////////////////////////////// test("sel star pruning") { @@ -346,6 +346,19 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { expectSql("select * from selstar where val='val_487'","487 val_487") } + test("map pruning with functions in between clause") { + sc.sql("drop table if exists mapsplitfunc") + sc.sql("drop table if exists mapsplitfunc_cached") + sc.sql("create table mapsplitfunc(k bigint, v string)") + sc.sql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + OVERWRITE INTO TABLE mapsplitfunc""") + sc.sql("create table mapsplitfunc_cached as select * from mapsplitfunc") + expectSql("""select count(*) from mapsplitfunc_cached + where month(from_unixtime(k)) between "1" and "12" """, Array[String]("500")) + expectSql("""select count(*) from mapsplitfunc_cached + where year(from_unixtime(k)) between "2013" and "2014" """, Array[String]("0")) + } + ////////////////////////////////////////////////////////////////////////////// // various data types ////////////////////////////////////////////////////////////////////////////// @@ -393,23 +406,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { Array[String]("23\t292", "36\t208")) } - ////////////////////////////////////////////////////////////////////////////// - // Test function based pruning Bug - ////////////////////////////////////////////////////////////////////////////// - - test("map pruning with functions in between clause") { - sc.sql("drop table if exists mapsplitfunc") - sc.sql("drop table if exists mapsplitfunc_cached") - sc.sql("create table mapsplitfunc(k bigint, v string)") - sc.sql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' - OVERWRITE INTO TABLE mapsplitfunc""") - sc.sql("create table mapsplitfunc_cached as select * from mapsplitfunc") - expectSql("""select count(*) from mapsplitfunc_cached - where month(from_unixtime(k)) between "1" and "12" """, Array[String]("500")) - expectSql("""select count(*) from mapsplitfunc_cached - where year(from_unixtime(k)) between "2013" and "2014" """, Array[String]("0")) - } - ////////////////////////////////////////////////////////////////////////////// // SharkContext APIs (e.g. sql2rdd, sql) ////////////////////////////////////////////////////////////////////////////// From ac8e7c8bb43057f3132f4e7b8f8465c41a7c5ea3 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 27 Sep 2013 13:54:29 -0700 Subject: [PATCH 076/331] exclude jetty from tachyon --- project/SharkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index d4940f2f..0706f25a 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -108,6 +108,6 @@ object SharkBuild extends Build { "junit" % "junit" % "4.10" % "test", "net.java.dev.jets3t" % "jets3t" % "0.9.0", "com.novocode" % "junit-interface" % "0.8" % "test") ++ - (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop) ) else None).toSeq + (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop, excludeNetty) ) else None).toSeq ) } From d6e67301d00b44fb52c7f6fb5da429ec2c7f1e9c Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 27 Sep 2013 15:50:49 -0700 Subject: [PATCH 077/331] exclude curator from tachyon --- project/SharkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 0706f25a..310f63f3 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -42,6 +42,7 @@ object SharkBuild extends Build { val excludeKyro = ExclusionRule(organization = "de.javakaffee") val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeCurator = ExclusionRule(organization = "org.apache.curator") def coreSettings = Defaults.defaultSettings ++ Seq( @@ -108,6 +109,6 @@ object SharkBuild extends Build { "junit" % "junit" % "4.10" % "test", "net.java.dev.jets3t" % "jets3t" % "0.9.0", "com.novocode" % "junit-interface" % "0.8" % "test") ++ - (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop, excludeNetty) ) else None).toSeq + (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop, excludeNetty, excludeCurator) ) else None).toSeq ) } From 384f5b39c2ba0e58b55242e70ce7083b031b5821 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 28 Sep 2013 17:00:46 -0700 Subject: [PATCH 078/331] Moved Spark's repository to its ASF github mirror. --- bin/dev/run-tests-from-scratch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/dev/run-tests-from-scratch b/bin/dev/run-tests-from-scratch index 1ed30b3b..7e3d940b 100755 --- a/bin/dev/run-tests-from-scratch +++ b/bin/dev/run-tests-from-scratch @@ -15,7 +15,7 @@ SHARK_PROJ_DIR_DEFAULT="$(cd `dirname $0`/../../; pwd)" SPARK_MEM_DEFAULT=4g SHARK_MASTER_MEM_DEFAULT=4g SPARK_KV_JAVA_OPTS_DEFAULT=("-Dspark.local.dir=/tmp " "-Dspark.kryoserializer.buffer.mb=10 ") -SPARK_GIT_URL_DEFAULT="https://github.com/mesos/spark.git" +SPARK_GIT_URL_DEFAULT="https://github.com/apache/incubator-spark.git" HIVE_GIT_URL_DEFAULT="https://github.com/amplab/hive.git -b shark-0.9" SPARK_HADOOP_VERSION_DEFAULT="1.0.4" SPARK_WITH_YARN_DEFAULT=false From 1f77cc5ab5ea13b16b0242e6546773d686728370 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 28 Sep 2013 17:01:57 -0700 Subject: [PATCH 079/331] Clone Spark into the spark folder for CI script (instead of incubator-spark). --- bin/dev/run-tests-from-scratch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/dev/run-tests-from-scratch b/bin/dev/run-tests-from-scratch index 7e3d940b..a60256e6 100755 --- a/bin/dev/run-tests-from-scratch +++ b/bin/dev/run-tests-from-scratch @@ -15,7 +15,7 @@ SHARK_PROJ_DIR_DEFAULT="$(cd `dirname $0`/../../; pwd)" SPARK_MEM_DEFAULT=4g SHARK_MASTER_MEM_DEFAULT=4g SPARK_KV_JAVA_OPTS_DEFAULT=("-Dspark.local.dir=/tmp " "-Dspark.kryoserializer.buffer.mb=10 ") -SPARK_GIT_URL_DEFAULT="https://github.com/apache/incubator-spark.git" +SPARK_GIT_URL_DEFAULT="https://github.com/apache/incubator-spark.git spark" HIVE_GIT_URL_DEFAULT="https://github.com/amplab/hive.git -b shark-0.9" SPARK_HADOOP_VERSION_DEFAULT="1.0.4" SPARK_WITH_YARN_DEFAULT=false From 938f508e22fcd4a748816bdf94e751d2c3c9cea5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 30 Sep 2013 00:43:11 -0700 Subject: [PATCH 080/331] Removed the extra runJob in cached table creation. --- src/main/scala/shark/execution/MemoryStoreSinkOperator.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index fc989885..10e99551 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -124,7 +124,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Force evaluate so the data gets put into Spark block manager. rdd.persist(storageLevel) - rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) val origRdd = rdd if (useUnionRDD) { From 0c825e72efe27787191cbba2ab7efbecea518116 Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Fri, 4 Oct 2013 07:50:51 -0700 Subject: [PATCH 081/331] Warnings now errors. Disabled cyclomatic complexity --- scalastyle-config.xml | 83 +++++++++++++++++++++++-------------------- 1 file changed, 45 insertions(+), 38 deletions(-) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index a65482e0..d3b75788 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -1,12 +1,19 @@ + + + + + + + Scalastyle standard configuration - - + + - + - - - - + + + + - + - + - + - - + + - + - + - - - - - - - - - + + + + + + + + + - + - - - - - - - - + + + + + + + + - + - + - + - - - + + + From 1637018c51419381dd5bafacdd6d1fafac802c02 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 4 Oct 2013 21:06:08 -0700 Subject: [PATCH 082/331] Add a SparkDDLTask for Shark-specific metastore operations. Includes support for ALTER TABLE RENAME TO . Currently, this is meant to be a dependent task on Hive's DDLTask. --- src/main/scala/shark/SharkDriver.scala | 4 +- .../scala/shark/execution/SparkDDLTask.scala | 70 +++++++++++++++++++ .../memstore2/MemoryMetadataManager.scala | 13 ++++ .../parse/SharkDDLSemanticAnalyzer.scala | 38 ++++++++-- 4 files changed, 118 insertions(+), 7 deletions(-) create mode 100644 src/main/scala/shark/execution/SparkDDLTask.scala diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 6a11aa2b..01bcbf3f 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -35,7 +35,8 @@ import org.apache.hadoop.util.StringUtils import shark.api.TableRDD import shark.api.QueryExecutionException -import shark.execution.{SharkExplainTask, SharkExplainWork, SparkTask, SparkWork} +import shark.execution.{SparkDDLTask, SparkDDLWork, SharkExplainTask, SharkExplainWork, SparkTask, + SparkWork} import shark.memstore2.ColumnarSerDe import shark.parse.{QueryContext, SharkSemanticAnalyzerFactory} @@ -62,6 +63,7 @@ private[shark] object SharkDriver extends LogHelper { // Task factory. Add Shark specific tasks. TaskFactory.taskvec.addAll(Seq( + new TaskFactory.taskTuple(classOf[SparkDDLWork], classOf[SparkDDLTask]), new TaskFactory.taskTuple(classOf[SparkWork], classOf[SparkTask]), new TaskFactory.taskTuple(classOf[SharkExplainWork], classOf[SharkExplainTask]))) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala new file mode 100644 index 00000000..1b19d018 --- /dev/null +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.ql.{Context, DriverContext} +import org.apache.hadoop.hive.ql.exec.{Task => HiveTask, TaskExecutionException} +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.plan._ +import org.apache.hadoop.hive.ql.plan.api.StageType + +import shark.{LogHelper, SharkEnv} +import shark.memstore2.{CacheType, MemoryMetadataManager} + + +private[shark] class SparkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { + // Used only for CREATE TABLE. + var cacheMode: CacheType.CacheType = _ +} + +private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelper { + + override def execute(driverContext: DriverContext): Int = { + val hiveMetadataDb = Hive.get(conf) + + work.ddlDesc match { + case alterTableDesc: AlterTableDesc => { + alterTable(hiveMetadataDb, alterTableDesc) + } + case _ => { + throw new UnsupportedOperationException( + "Shark does not require a Spark DDL task for: " + work.ddlDesc.getClass.getName) + } + } + + // Hive's task runner expects a '0' return value to indicate success and exceptions on + // failure. + return 0 + } + + def alterTable( + hiveMetadataDb: Hive, + alterTableDesc: AlterTableDesc) { + val oldName = alterTableDesc.getOldName + val newName = alterTableDesc.getNewName + SharkEnv.memoryMetadataManager.rename(oldName, newName) + } + + override def getType = StageType.DDL + + override def getName = "DDL-SPARK" + + override def localizeMRTmpFilesImpl(ctx: Context) = Unit +} diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index c180dd40..ed6efa49 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -52,6 +52,19 @@ class MemoryMetadataManager { _keyToStats.get(key.toLowerCase) } + def rename(oldKey: String, newKey: String) { + if (contains(oldKey)) { + val oldKeyToLowerCase = oldKey.toLowerCase + val newKeyToLowerCase = newKey.toLowerCase + + val statsValueEntry = _keyToStats.remove(oldKeyToLowerCase).get + val rddValueEntry = _keyToRdd.remove(oldKeyToLowerCase).get + + _keyToStats.put(newKeyToLowerCase, statsValueEntry) + _keyToRdd.put(newKeyToLowerCase, rddValueEntry) + } + } + /** * Find all keys that are strings. Used to drop tables after exiting. */ diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index a43a4975..fdb298ca 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -1,24 +1,50 @@ package shark.parse +import scala.collection.JavaConversions._ + import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.exec.TaskFactory import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, HiveParser} +import org.apache.hadoop.hive.ql.plan.DDLWork import org.apache.spark.rdd.{UnionRDD, RDD} +import shark.execution.SparkDDLWork import shark.{LogHelper, SharkEnv} +import shark.memstore2.MemoryMetadataManager class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) with LogHelper { - override def analyzeInternal(node: ASTNode): Unit = { - super.analyzeInternal(node) - //handle drop table query - if (node.getToken().getType() == HiveParser.TOK_DROPTABLE) { - SharkEnv.unpersist(getTableName(node)) + override def analyzeInternal(astNode: ASTNode): Unit = { + super.analyzeInternal(astNode) + + astNode.getToken.getType match { + case HiveParser.TOK_ALTERTABLE_RENAME => { + analyzeAlterTableRename(astNode) + } + case _ => Unit } } + private def analyzeAlterTableRename(astNode: ASTNode) { + val oldTableName = getTableName(astNode) + val newTableName = BaseSemanticAnalyzer.getUnescapedName( + astNode.getChild(1).asInstanceOf[ASTNode]) + + // Hive's DDLSemanticAnalyzer#AnalyzeInternal() will only populate rootTasks with a DDLTask + // and DDLWork that contains an AlterTableDesc. + assert(rootTasks.size == 1) + val ddlTask = rootTasks.head + val ddlWork = ddlTask.getWork + assert(ddlWork.isInstanceOf[DDLWork]) + + val alterTableDesc = ddlWork.asInstanceOf[DDLWork].getAlterTblDesc + val sparkDDLWork = new SparkDDLWork(alterTableDesc) + ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + } + private def getTableName(node: ASTNode): String = { BaseSemanticAnalyzer.getUnescapedName(node.getChild(0).asInstanceOf[ASTNode]) } -} \ No newline at end of file +} From df6bb1bd6aad8831922cb9b23fc81b6c84ed5e04 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 4 Oct 2013 21:13:19 -0700 Subject: [PATCH 083/331] Unit test for ALTER TABLE RENAME TO on cached tables. --- src/test/scala/shark/SQLSuite.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 747171b6..15a3fe9f 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -222,6 +222,16 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // cache DDL ////////////////////////////////////////////////////////////////////////////// + test("rename cached table") { + sc.runSql("drop table if exists test_oldname_cached") + sc.runSql("drop table if exists test_rename") + sc.runSql("create table test_oldname_cached as select * from test") + sc.runSql("alter table test_oldname_cached rename to test_rename") + assert(!SharkEnv.memoryMetadataManager.contains("test_oldname_cached")) + assert(SharkEnv.memoryMetadataManager.contains("test_rename")) + expectSql("select count(*) from test_rename", "500") + } + test("insert into cached tables") { sc.runSql("drop table if exists test1_cached") sc.runSql("create table test1_cached as select * from test") From 0a0de4e9af4cf5586a7127239847f38bd344b491 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 4 Oct 2013 21:18:10 -0700 Subject: [PATCH 084/331] Fix merge mistake in SharkDDLSemanticAnalyzer. --- src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index fdb298ca..f14eda49 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -20,6 +20,9 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) super.analyzeInternal(astNode) astNode.getToken.getType match { + case HiveParser.TOK_DROPTABLE => { + SharkEnv.unpersist(getTableName(astNode)) + } case HiveParser.TOK_ALTERTABLE_RENAME => { analyzeAlterTableRename(astNode) } From 186d573539f734974a375d4d7002b7a1c013904d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 5 Oct 2013 16:40:40 -0700 Subject: [PATCH 085/331] Get merges to compile. --- src/main/scala/shark/execution/SparkDDLTask.scala | 9 ++++++--- .../scala/shark/memstore2/MemoryMetadataManager.scala | 4 ++-- .../scala/shark/parse/SharkDDLSemanticAnalyzer.scala | 2 +- src/test/scala/shark/SQLSuite.scala | 1 + 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 25eb56ec..5ed16fe6 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -100,13 +100,16 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab partitionColumns, partitionColumnToValue) SharkEnv.memoryMetadataManager.dropHivePartition(tableName, keyStr) } + } def alterTable( hiveMetadataDb: Hive, alterTableDesc: AlterTableDesc) { - val oldName = alterTableDesc.getOldName - val newName = alterTableDesc.getNewName - SharkEnv.memoryMetadataManager.rename(oldName, newName) + if (alterTableDesc.getOp() == AlterTableDesc.AlterTableTypes.RENAME) { + val oldName = alterTableDesc.getOldName + val newName = alterTableDesc.getNewName + SharkEnv.memoryMetadataManager.rename(oldName, newName) + } } override def getType = StageType.DDL diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index a08478df..9593f836 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -118,10 +118,10 @@ class MemoryMetadataManager { val newKeyToLowerCase = newKey.toLowerCase val statsValueEntry = _keyToStats.remove(oldKeyToLowerCase).get - val rddValueEntry = _keyToRdd.remove(oldKeyToLowerCase).get + val rddValueEntry = _keyToMemoryTable.remove(oldKeyToLowerCase).get _keyToStats.put(newKeyToLowerCase, statsValueEntry) - _keyToRdd.put(newKeyToLowerCase, rddValueEntry) + _keyToMemoryTable.put(newKeyToLowerCase, rddValueEntry) } } diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index d8e4fd59..f3d2d4b1 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -34,7 +34,7 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) alterTableDropParts(ast) } case HiveParser.TOK_ALTERTABLE_RENAME => { - analyzeAlterTableRename(astNode) + analyzeAlterTableRename(ast) } case _ => Unit } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 69f5ddef..c88bb3d8 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -242,6 +242,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("create table new_table_cached(key string, value string)") sc.runSql("insert into table new_table_cached select * from test where key > -1 limit 499") expectSql("select count(*) from new_table_cached", "499") + } test("rename cached table") { sc.runSql("drop table if exists test_oldname_cached") From c231cf71efad74304417c00dc9241ef2e9a408a0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 5 Oct 2013 16:41:54 -0700 Subject: [PATCH 086/331] Make SparkDDLTask#alterTable() more general. + Only create a SparkDDLTask for cached table ops. --- .../scala/shark/execution/SparkDDLTask.scala | 8 ++++-- .../parse/SharkDDLSemanticAnalyzer.scala | 28 ++++++++++--------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 1b19d018..f54f169f 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -57,9 +57,11 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab def alterTable( hiveMetadataDb: Hive, alterTableDesc: AlterTableDesc) { - val oldName = alterTableDesc.getOldName - val newName = alterTableDesc.getNewName - SharkEnv.memoryMetadataManager.rename(oldName, newName) + if (alterTableDesc.getOp() == AlterTableDesc.AlterTableTypes.RENAME) { + val oldName = alterTableDesc.getOldName + val newName = alterTableDesc.getNewName + SharkEnv.memoryMetadataManager.rename(oldName, newName) + } } override def getType = StageType.DDL diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index f14eda49..2db2026b 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -32,19 +32,21 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) private def analyzeAlterTableRename(astNode: ASTNode) { val oldTableName = getTableName(astNode) - val newTableName = BaseSemanticAnalyzer.getUnescapedName( - astNode.getChild(1).asInstanceOf[ASTNode]) - - // Hive's DDLSemanticAnalyzer#AnalyzeInternal() will only populate rootTasks with a DDLTask - // and DDLWork that contains an AlterTableDesc. - assert(rootTasks.size == 1) - val ddlTask = rootTasks.head - val ddlWork = ddlTask.getWork - assert(ddlWork.isInstanceOf[DDLWork]) - - val alterTableDesc = ddlWork.asInstanceOf[DDLWork].getAlterTblDesc - val sparkDDLWork = new SparkDDLWork(alterTableDesc) - ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + if (SharkEnv.memoryMetadataManager.contains(oldTableName)) { + val newTableName = BaseSemanticAnalyzer.getUnescapedName( + astNode.getChild(1).asInstanceOf[ASTNode]) + + // Hive's DDLSemanticAnalyzer#AnalyzeInternal() will only populate rootTasks with a DDLTask + // and DDLWork that contains an AlterTableDesc. + assert(rootTasks.size == 1) + val ddlTask = rootTasks.head + val ddlWork = ddlTask.getWork + assert(ddlWork.isInstanceOf[DDLWork]) + + val alterTableDesc = ddlWork.asInstanceOf[DDLWork].getAlterTblDesc + val sparkDDLWork = new SparkDDLWork(alterTableDesc) + ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + } } private def getTableName(node: ASTNode): String = { From 9c7093968fda938743a6a65d17e1326c59cb0172 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 00:54:14 -0700 Subject: [PATCH 087/331] Add a CachePolicy abstract class, with an LRUCachePolicy implementation. --- .../scala/shark/memstore2/CachePolicy.scala | 78 +++++++++++++++++++ 1 file changed, 78 insertions(+) create mode 100644 src/main/scala/shark/memstore2/CachePolicy.scala diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala new file mode 100644 index 00000000..c436dda6 --- /dev/null +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -0,0 +1,78 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import com.google.common.cache._ + +import scala.collection.JavaConversions._ + + +private[shark] abstract class CachePolicy[K, V] { + + var maxCacheSize: Long = _ + + var loadFunction: (K => V) = _ + + var evictionFunction: ((K, V) => Unit) = _ + + def initialize(): Unit + + def notifyPut(key: K, value: V): Unit + + def notifyRemove(key: K, value: V): Unit + + def notifyGet(key: K): Unit + + def getKeysOfCachedEntries: Seq[K] +} + +private[shark] class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { + + var cache: LoadingCache[K, V] = _ + + override def initialize(): Unit = { + var builder = CacheBuilder.newBuilder().maximumSize(maxCacheSize) + + val removalListener = + new RemovalListener[K, V] { + def onRemoval(removal: RemovalNotification[K, V]): Unit = + evictionFunction(removal.getKey, removal.getValue) + } + val cacheLoader = + new CacheLoader[K, V] { + def load(key: K): V = loadFunction(key) + } + + cache = builder.removalListener(removalListener).build(cacheLoader) + } + + override def notifyPut(key: K, value: V): Unit = { + cache.put(key, value) + } + + override def notifyRemove(key: K, value: V): Unit = { + cache.invalidate(key, value) + } + + override def notifyGet(key: K): Unit = { + cache.get(key) + } + + protected def getKeysOfCachedEntries: Seq[K] = cache.asMap.keySet.toSeq + +} From faa84aa335c523f4359f77da2bbd0cf01b9cde3e Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 00:54:44 -0700 Subject: [PATCH 088/331] Have separate MemoryTable classes for Hive-partitioned tables and non-partitioned tables. --- .../scala/shark/memstore2/MemoryTable.scala | 71 ++++++++++--------- 1 file changed, 39 insertions(+), 32 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index bd7081a8..0ae66d07 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -17,10 +17,10 @@ package shark.memstore2 -import java.util.{HashMap => JavaHashMap} +import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.Map +import scala.collection.mutable.ConcurrentMap import org.apache.spark.rdd.RDD @@ -36,48 +36,55 @@ import org.apache.spark.rdd.RDD * such as HivePartitionedTable or TachyonTable, subclass it. For now, there isn't * too much metadata to track, so it should be okay to have a single MemoryTable. */ + +private[shark] abstract class Table(val tableName: String, val cacheMode: CacheType.CacheType) + private[shark] class MemoryTable( val tableName: String, - val isHivePartitioned: Boolean) { + val cacheMode: CacheType.CacheType) + extends Table(tableName, cacheMode) { - // Should only be used if the table is not Hive-partitioned. - private var _tableRDD: RDD[_] = _ + // RDD that contains the contents of this table. + var tableRDD: RDD[_] = _ +} - // Should only be used if a cached table is Hive-partitioned. - private var _keyToHivePartitions: Map[String, RDD[_]] = _ +private[shark] +class PartitionedMemoryTable( + val tableName: String, + val cacheMode: CacheType.CacheType) + extends Table(tableName, cacheMode) { + + // A map from the Hive-partition key to the RDD that contains contents of that partition. + private var _keyToPartitions: ConcurrentMap[String, RDD[_]] = + new ConcurrentJavaHashMap[String, RDD[_]]() - // CacheMode for the table. - // This is common to all Hive-partitions (if applicable). - var cacheMode: CacheType.CacheType = _ + // The eviction policy for this table's cached Hive-partitions. An example of how this + // can be set from the CLI: + // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. + private var _partitionCachePolicy: CachePolicy[String, RDD[_]] = _ - def tableRDD: RDD[_] = { - assert ( - !isHivePartitioned, - "Table " + tableName + " is Hive-partitioned. Use MemoryTableDesc::hivePartitionRDDs() " + - "to get RDDs corresponding to partition columns" - ) - return _tableRDD + def getPartition(partitionKey: String): Option[RDD[_]] = { + val rddFound = _keyToPartitions.get(partitionKey) + if (rddFound.isDefined) _partitionCachePolicy.notifyGet(partitionKey) + return rddFound } - def tableRDD_= (value: RDD[_]) { - assert( - !isHivePartitioned, - "Table " + tableName + " is Hive-partitioned. Pass in a map of pairs " + - "to the 'keyToHivePartitions =' setter." - ) - _tableRDD = value + def putPartition(partitionKey: String, rdd: RDD[_]): Option[RDD[_]] = { + _partitionCachePolicy.notifyPut(partitionKey, rdd) + _keyToPartitions.put(partitionKey, rdd) } - def keyToHivePartitions: Map[String, RDD[_]] = { - assert(isHivePartitioned, - "Table " + tableName + " is not Hive-partitioned. Use tableRDD() to get its RDD.") - _keyToHivePartitions + def removePartition(partitionKey: String): Option[RDD[_]] = { + val rddRemoved = _keyToPartitions.remove(partitionKey) + if (rddRemoved.isDefined) _partitionCachePolicy.notifyRemove(partitionKey, rddRemoved.get) + return rddRemoved } - def keyToHivePartitions_= (value: Map[String, RDD[_]]) { - assert(isHivePartitioned, - "Table " + tableName + " is not Hive-partitioned. Use 'tableRDD =' to set the RDD.") - _keyToHivePartitions = value + def partitionCachePolicy_= (value: String) { + _partitionCachePolicy = + Class.forName(value).newInstance.asInstanceOf[CachePolicy[String, RDD[_]]] } + + def partitionCachePolicy: CachePolicy[String, RDD[_]] = _partitionCachePolicy } From e70c194e5d95c4811950eda074bb25f146c35d48 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 01:14:07 -0700 Subject: [PATCH 089/331] Make MemoryMetadataManager compliant with new (Shark) Table API. --- .../scala/shark/memstore2/CachePolicy.scala | 3 +- .../memstore2/MemoryMetadataManager.scala | 78 +++++-------------- .../scala/shark/memstore2/MemoryTable.scala | 21 ++++- 3 files changed, 39 insertions(+), 63 deletions(-) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index c436dda6..cec78b53 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -46,7 +46,7 @@ private[shark] class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolic var cache: LoadingCache[K, V] = _ override def initialize(): Unit = { - var builder = CacheBuilder.newBuilder().maximumSize(maxCacheSize) + var builder = CacheBuilder.newBuilder().maximumSize(maxCacheSize) val removalListener = new RemovalListener[K, V] { @@ -74,5 +74,4 @@ private[shark] class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolic } protected def getKeysOfCachedEntries: Seq[K] = cache.asMap.keySet.toSeq - } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 9593f836..e4c6bc43 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -35,74 +35,40 @@ import shark.SharkEnv // addCreatedTable(), which should also take in a Hive DB (metastore) name. class MemoryMetadataManager { - private val _keyToMemoryTable: ConcurrentMap[String, MemoryTable] = - new ConcurrentHashMap[String, MemoryTable]() + private val _keyToMemoryTable: ConcurrentMap[String, Table] = + new ConcurrentHashMap[String, Table]() // TODO(harvey): Support stats for cached Hive-partitioned tables. private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] def add(key: String, isHivePartitioned: Boolean, cacheMode: CacheType.CacheType) { - val memoryTable = new MemoryTable(key.toLowerCase, isHivePartitioned) - if (isHivePartitioned) { - memoryTable.keyToHivePartitions = new JavaHashMap[String, RDD[_]]() - } - memoryTable.cacheMode = cacheMode + var memoryTable = + if (isHivePartitioned) { + new PartitionedMemoryTable(key.toLowerCase, cacheMode) + } else { + new MemoryTable(key.toLowerCase, cacheMode) + } _keyToMemoryTable(key.toLowerCase) = memoryTable } def getCacheMode(key: String): CacheType.CacheType = { _keyToMemoryTable.get(key.toLowerCase) match { - case Some(memoryTable) => return memoryTable.cacheMode + case Some(table) => return table.cacheMode case _ => return CacheType.NONE } } def isHivePartitioned(key: String): Boolean = { _keyToMemoryTable.get(key.toLowerCase) match { - case Some(memoryTable) => return memoryTable.isHivePartitioned + case Some(table) => return Table.isHivePartitioned((table)) case None => return false } } def contains(key: String): Boolean = _keyToMemoryTable.contains(key.toLowerCase) - def containsHivePartition(key: String, partitionColumnValues: String): Boolean = { - val containsTable = _keyToMemoryTable.contains(key.toLowerCase) - return (containsTable && - _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.contains(partitionColumnValues)) - } - - def put(key: String, rdd: RDD[_]) { - val memoryTable = _keyToMemoryTable(key.toLowerCase) - memoryTable.tableRDD = rdd - } - - def putHivePartition( - key: String, - partitionColumnValues: String, - rdd: RDD[_]) { - val keyToHivePartitions = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions - keyToHivePartitions(partitionColumnValues) = rdd - } - - def dropHivePartition(key: String, partitionColumnValues: String) { - val keyToHivePartitions = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions - val rdd = keyToHivePartitions.remove(partitionColumnValues) - unpersistRDD(rdd.get) - } - - def get(key: String): Option[RDD[_]] = { - _keyToMemoryTable.get(key.toLowerCase) match { - case Some(memoryTable) => return Some(memoryTable.tableRDD) - case _ => return None - } - } - - def getHivePartition(key: String, partitionColumnValues: String): Option[RDD[_]] = { - val keyToHivePartitions = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions - keyToHivePartitions.get(partitionColumnValues) - } + def get(key: String): Option[Table] = _keyToMemoryTable.get(key.toLowerCase) def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { _keyToStats.put(key.toLowerCase, stats) @@ -132,15 +98,6 @@ class MemoryMetadataManager { _keyToMemoryTable.keys.collect { case k: String => k } toSeq } - def getStorageLevel(key: String): StorageLevel = { - if (isHivePartitioned(key.toLowerCase)) { - val hivePartitionRDDs = _keyToMemoryTable(key.toLowerCase).keyToHivePartitions.values - return RDDUtils.getStorageLevelOfCachedRDDs(hivePartitionRDDs.toSeq) - } else { - return RDDUtils.getStorageLevelOfCachedRDD(get(key.toLowerCase).get) - } - } - /** * Used to drop a table from the Spark in-memory cache and/or disk. All metadata * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) tracked by Shark is deleted @@ -152,18 +109,19 @@ class MemoryMetadataManager { * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ def unpersist(key: String): Option[RDD[_]] = { - def unpersistMemoryTable(memoryTable: MemoryTable): Option[RDD[_]] = { + def unpersistTable(table: Table): Option[RDD[_]] = { var unpersistedRDD: Option[RDD[_]] = None - if (memoryTable.isHivePartitioned) { + if (Table.isHivePartitioned(table)) { + val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] // unpersist() all RDDs for all Hive-partitions. - val unpersistedRDDs = memoryTable.keyToHivePartitions.values.map( + val unpersistedRDDs = partitionedTable.getAllPartitions.map( rdd => unpersistRDD(rdd)).asInstanceOf[Seq[RDD[Any]]] if (unpersistedRDDs.size > 0) { val unionedRDD = new UnionRDD(unpersistedRDDs.head.context, unpersistedRDDs) unpersistedRDD = Some(unionedRDD) } } else { - unpersistedRDD = Some(unpersistRDD(memoryTable.tableRDD)) + unpersistedRDD = Some(unpersistRDD(table.asInstanceOf[MemoryTable].tableRDD)) } return unpersistedRDD } @@ -171,8 +129,8 @@ class MemoryMetadataManager { // Remove MemoryTable's entry from Shark metadata. _keyToStats.remove(key.toLowerCase) - val memoryTableValue: Option[MemoryTable] = _keyToMemoryTable.remove(key.toLowerCase) - return memoryTableValue.flatMap(unpersistMemoryTable(_)) + val tableValue: Option[Table] = _keyToMemoryTable.remove(key.toLowerCase) + return tableValue.flatMap(unpersistTable(_)) } def unpersistRDD(rdd: RDD[_]): RDD[_] = { diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 0ae66d07..82dcc4ca 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -23,6 +23,9 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ConcurrentMap import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import shark.execution.RDDUtils /** @@ -37,7 +40,15 @@ import org.apache.spark.rdd.RDD * too much metadata to track, so it should be okay to have a single MemoryTable. */ -private[shark] abstract class Table(val tableName: String, val cacheMode: CacheType.CacheType) +private[shark] abstract class Table(val tableName: String, val cacheMode: CacheType.CacheType) { + def getStorageLevel: StorageLevel +} + +object Table { + + def isHivePartitioned(table: Table) = table.isInstanceOf[PartitionedMemoryTable] + +} private[shark] class MemoryTable( @@ -47,6 +58,8 @@ class MemoryTable( // RDD that contains the contents of this table. var tableRDD: RDD[_] = _ + + override def getStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfCachedRDD(tableRDD) } private[shark] @@ -87,4 +100,10 @@ class PartitionedMemoryTable( } def partitionCachePolicy: CachePolicy[String, RDD[_]] = _partitionCachePolicy + + def getAllPartitions = _keyToPartitions.values.toSeq + + def getAllPartitionKeys = _keyToPartitions.keys.toSeq + + def getStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfCachedRDDs(getAllPartitions) } From 0d9cac310483f4cafeb2f743f4d61ed72880cb04 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 03:16:07 -0700 Subject: [PATCH 090/331] Directly use MemoryTable API to access RDDs in relevant classes, instead of going through MemoryMetadataManager. --- .../execution/MemoryStoreSinkOperator.scala | 24 +++--- .../scala/shark/execution/SparkDDLTask.scala | 30 ++++--- .../shark/execution/TableScanOperator.scala | 7 +- .../scala/shark/memstore2/CachePolicy.scala | 44 +++++++--- .../memstore2/MemoryMetadataManager.scala | 83 +++++++++++-------- .../scala/shark/memstore2/MemoryTable.scala | 37 +++++---- .../parse/SharkDDLSemanticAnalyzer.scala | 6 +- .../shark/parse/SharkSemanticAnalyzer.scala | 7 +- 8 files changed, 147 insertions(+), 91 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index c73948a3..5eba8ec3 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -127,14 +127,15 @@ class MemoryStoreSinkOperator extends TerminalOperator { val origRdd = rdd if (useUnionRDD) { - val oldRdd: Option[RDD[_]] = + var oldRdd: Option[RDD[_]] = if (isHivePartitioned) { - SharkEnv.memoryMetadataManager.getHivePartition(tableName, hivePartitionKey) + SharkEnv.memoryMetadataManager.getPartitionedTable(tableName) + .flatMap(_.getPartition(hivePartitionKey)) } else { - SharkEnv.memoryMetadataManager.get(tableName) + SharkEnv.memoryMetadataManager.getMemoryTable(tableName).map(_.tableRDD) } // If this is an insert, find the existing RDD and create a union of the two, and then - // put the union into the meta data tracker. + // put the union into the metadata tracker. rdd = oldRdd match { case Some(definedRdd) => rdd.union(oldRdd.get.asInstanceOf[RDD[TablePartition]]) // The oldRdd can be missing if this is an INSERT into a new Hive-partition. @@ -144,16 +145,17 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Run a job on the original RDD to force it to go into cache. origRdd.context.runJob(origRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } + if (isHivePartitioned) { - SharkEnv.memoryMetadataManager.putHivePartition(tableName, hivePartitionKey, rdd) - rdd.setName(tableName + "(" + hivePartitionKey + ")") - } else { - if (!SharkEnv.memoryMetadataManager.contains(tableName)) { - // This is a CTAS. Add a new table entry to the Shark metadata. - SharkEnv.memoryMetadataManager.add(tableName, false /* isHivePartitioned */, cacheMode) + SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).foreach{ table => + table.putPartition(hivePartitionKey, rdd) + rdd.setName(tableName + "(" + hivePartitionKey + ")") } - SharkEnv.memoryMetadataManager.put(tableName, rdd) + } else { + val table = SharkEnv.memoryMetadataManager.getMemoryTable(tableName).getOrElse( + SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode)) rdd.setName(tableName) + table.tableRDD = rdd } // Report remaining memory. diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 5ed16fe6..5cd8ec6a 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -67,11 +67,17 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab hiveMetadataDb: Hive, createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { - val isHivePartitioned = (createTblDesc.getPartCols.size > 0) val tableName = createTblDesc.getTableName - SharkEnv.memoryMetadataManager.add(tableName, isHivePartitioned, cacheMode) - if (!isHivePartitioned) { - SharkEnv.memoryMetadataManager.put(tableName, new EmptyRDD(SharkEnv.sc)) + val isHivePartitioned = (createTblDesc.getPartCols.size > 0) + if (isHivePartitioned) { + val tblProps = createTblDesc.getTblProps + val cachePolicyStr = tblProps.get("shark.cache.partition.cachePolicy") + val maxCacheSize = tblProps.get("shark.cache.partition.cachePolicy.maxSize").toLong + SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( + tableName, cacheMode, cachePolicyStr, maxCacheSize) + } else { + val newTable = SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode) + newTable.tableRDD = new EmptyRDD(SharkEnv.sc) } } @@ -79,26 +85,28 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab hiveMetadataDb: Hive, addPartitionDesc: AddPartitionDesc) { val tableName = addPartitionDesc.getTableName - val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionColumns = table.getPartCols.map(_.getName) + val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val partitionColumns = hiveTable.getPartCols.map(_.getName) val partitionColumnToValue = addPartitionDesc.getPartSpec val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( partitionColumns, partitionColumnToValue) - SharkEnv.memoryMetadataManager.putHivePartition(tableName, keyStr, new EmptyRDD(SharkEnv.sc)) + val partitionedTableOpt = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName) + partitionedTableOpt.map(_.putPartition(keyStr, new EmptyRDD(SharkEnv.sc))) } def dropTable( hiveMetadataDb: Hive, dropTableDesc: DropTableDesc) { val tableName = dropTableDesc.getTableName - val table = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionColumns = table.getPartCols.map(_.getName) + val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val partitionColumns = hiveTable.getPartCols.map(_.getName) val partSpecs = dropTableDesc.getPartSpecs for (partSpec <- partSpecs) { val partitionColumnToValue = partSpec.getPartSpecWithoutOperator val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( partitionColumns, partitionColumnToValue) - SharkEnv.memoryMetadataManager.dropHivePartition(tableName, keyStr) + val partitionedTableOpt = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName) + partitionedTableOpt.map(_.removePartition(keyStr)) } } @@ -108,7 +116,7 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab if (alterTableDesc.getOp() == AlterTableDesc.AlterTableTypes.RENAME) { val oldName = alterTableDesc.getOldName val newName = alterTableDesc.getNewName - SharkEnv.memoryMetadataManager.rename(oldName, newName) + SharkEnv.memoryMetadataManager.renameTable(oldName, newName) } } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index d0485362..43bdac64 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -121,7 +121,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). - if (!SharkEnv.memoryMetadataManager.contains(tableKey)) { + if (!SharkEnv.memoryMetadataManager.containsTable(tableKey)) { logError("""|Table %s not found in block manager. |Are you trying to access a cached table from a Shark session other than |the one in which it was created?""".stripMargin.format(tableKey)) @@ -131,7 +131,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // Get the union of RDDs repesenting the selected Hive partition(s). makeCachedPartitionRDD(tableKey, parts) } else { - val rdd = SharkEnv.memoryMetadataManager.get(tableKey).get + val rdd = SharkEnv.memoryMetadataManager.getMemoryTable(tableKey).get.tableRDD logInfo("Loading table " + tableKey + " from Spark block manager") createPrunedRdd(tableKey, rdd) } @@ -266,7 +266,8 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } }.toArray val partKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partColumns, partSpec) - val hivePartitionRDD = SharkEnv.memoryMetadataManager.getHivePartition(tableKey, partKeyStr) + val hivePartitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableKey).get + val hivePartitionRDD = hivePartitionedTable.getPartition(partKeyStr) val serializedHconf = XmlSerializer.serialize(localHconf, localHconf) hivePartitionRDD.get.mapPartitions { iter => diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index cec78b53..d9cc52d4 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -24,54 +24,76 @@ import scala.collection.JavaConversions._ private[shark] abstract class CachePolicy[K, V] { - var maxCacheSize: Long = _ + protected var maxSize: Long = _ - var loadFunction: (K => V) = _ + protected var loadFunc: (K => V) = _ - var evictionFunction: ((K, V) => Unit) = _ + protected var evictionFunc: (K, V) => Unit = _ - def initialize(): Unit + def initialize( + maxSize: Long, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit + ): Unit = { + this.maxSize = maxSize + this.loadFunc = loadFunc + this.evictionFunc = evictionFunc + } def notifyPut(key: K, value: V): Unit - + def notifyRemove(key: K, value: V): Unit def notifyGet(key: K): Unit def getKeysOfCachedEntries: Seq[K] + + def getMaxSize = maxSize } private[shark] class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { + var isInitialized = false var cache: LoadingCache[K, V] = _ - override def initialize(): Unit = { - var builder = CacheBuilder.newBuilder().maximumSize(maxCacheSize) + override def initialize( + maxSize: Long, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit + ): Unit = { + super.initialize(maxSize, loadFunc, evictionFunc) + + var builder = CacheBuilder.newBuilder().maximumSize(maxSize) val removalListener = new RemovalListener[K, V] { - def onRemoval(removal: RemovalNotification[K, V]): Unit = - evictionFunction(removal.getKey, removal.getValue) + def onRemoval(removal: RemovalNotification[K, V]): Unit = { + evictionFunc(removal.getKey, removal.getValue) + } } val cacheLoader = new CacheLoader[K, V] { - def load(key: K): V = loadFunction(key) + def load(key: K): V = loadFunc(key) } cache = builder.removalListener(removalListener).build(cacheLoader) + isInitialized = true } override def notifyPut(key: K, value: V): Unit = { + assert(isInitialized, "LRUCachePolicy must be initialize()'d.") cache.put(key, value) } override def notifyRemove(key: K, value: V): Unit = { + assert(isInitialized, "LRUCachePolicy must be initialize()'d.") cache.invalidate(key, value) } override def notifyGet(key: K): Unit = { + assert(isInitialized, "LRUCachePolicy must be initialize()'d.") cache.get(key) } - protected def getKeysOfCachedEntries: Seq[K] = cache.asMap.keySet.toSeq + override def getKeysOfCachedEntries: Seq[K] = cache.asMap.keySet.toSeq } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index e4c6bc43..00776351 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -31,44 +31,54 @@ import shark.SharkConfVars import shark.SharkEnv -// TODO(harvey): Redo the interfaces to this class. For example, add() could be renamed to -// addCreatedTable(), which should also take in a Hive DB (metastore) name. class MemoryMetadataManager { - private val _keyToMemoryTable: ConcurrentMap[String, Table] = + private val _keyToTable: ConcurrentMap[String, Table] = new ConcurrentHashMap[String, Table]() // TODO(harvey): Support stats for cached Hive-partitioned tables. private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - def add(key: String, isHivePartitioned: Boolean, cacheMode: CacheType.CacheType) { - var memoryTable = - if (isHivePartitioned) { - new PartitionedMemoryTable(key.toLowerCase, cacheMode) - } else { - new MemoryTable(key.toLowerCase, cacheMode) - } - _keyToMemoryTable(key.toLowerCase) = memoryTable + def createMemoryTable( + tableName: String, + cacheMode: CacheType.CacheType + ): MemoryTable = { + var newTable = new MemoryTable(tableName.toLowerCase, cacheMode) + _keyToTable.put(tableName.toLowerCase, newTable) + return newTable } - - def getCacheMode(key: String): CacheType.CacheType = { - _keyToMemoryTable.get(key.toLowerCase) match { - case Some(table) => return table.cacheMode - case _ => return CacheType.NONE - } + + def createPartitionedMemoryTable( + tableName: String, + cacheMode: CacheType.CacheType, + cachePolicyStr: String, + cachePolicyMaxSize: Long + ): PartitionedMemoryTable = { + var newTable = new PartitionedMemoryTable(tableName.toLowerCase, cacheMode) + newTable.setPartitionCachePolicy(cachePolicyStr, cachePolicyMaxSize) + _keyToTable.put(tableName.toLowerCase, newTable) + return newTable } - def isHivePartitioned(key: String): Boolean = { - _keyToMemoryTable.get(key.toLowerCase) match { - case Some(table) => return Table.isHivePartitioned((table)) + def isHivePartitioned(tableName: String): Boolean = { + _keyToTable.get(tableName.toLowerCase) match { + case Some(table) => return table.isInstanceOf[PartitionedMemoryTable] case None => return false } } - def contains(key: String): Boolean = _keyToMemoryTable.contains(key.toLowerCase) + def containsTable(tableName: String): Boolean = _keyToTable.contains(tableName.toLowerCase) - def get(key: String): Option[Table] = _keyToMemoryTable.get(key.toLowerCase) + def getTable(tableName: String): Option[Table] = _keyToTable.get(tableName.toLowerCase) + + def getMemoryTable(tableName: String): Option[MemoryTable] = { + _keyToTable.get(tableName.toLowerCase).asInstanceOf[Option[MemoryTable]] + } + + def getPartitionedTable(tableName: String): Option[PartitionedMemoryTable] = { + _keyToTable.get(tableName.toLowerCase).asInstanceOf[Option[PartitionedMemoryTable]] + } def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { _keyToStats.put(key.toLowerCase, stats) @@ -78,16 +88,17 @@ class MemoryMetadataManager { _keyToStats.get(key.toLowerCase) } - def rename(oldKey: String, newKey: String) { - if (contains(oldKey)) { - val oldKeyToLowerCase = oldKey.toLowerCase - val newKeyToLowerCase = newKey.toLowerCase + def renameTable(oldName: String, newName: String) { + if (containsTable(oldName)) { + val lowerCaseOldName = oldName.toLowerCase + val lowerCaseNewName = newName.toLowerCase - val statsValueEntry = _keyToStats.remove(oldKeyToLowerCase).get - val rddValueEntry = _keyToMemoryTable.remove(oldKeyToLowerCase).get + val statsValueEntry = _keyToStats.remove(lowerCaseOldName).get + val tableValueEntry = _keyToTable.remove(lowerCaseOldName).get + tableValueEntry.tableName = lowerCaseNewName - _keyToStats.put(newKeyToLowerCase, statsValueEntry) - _keyToMemoryTable.put(newKeyToLowerCase, rddValueEntry) + _keyToStats.put(lowerCaseNewName, statsValueEntry) + _keyToTable.put(lowerCaseNewName, tableValueEntry) } } @@ -95,7 +106,7 @@ class MemoryMetadataManager { * Find all keys that are strings. Used to drop tables after exiting. */ def getAllKeyStrings(): Seq[String] = { - _keyToMemoryTable.keys.collect { case k: String => k } toSeq + _keyToTable.keys.collect { case k: String => k } toSeq } /** @@ -108,10 +119,12 @@ class MemoryMetadataManager { * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ - def unpersist(key: String): Option[RDD[_]] = { + def unpersist(tableName: String): Option[RDD[_]] = { + val lowerCaseTableName = tableName.toLowerCase + def unpersistTable(table: Table): Option[RDD[_]] = { var unpersistedRDD: Option[RDD[_]] = None - if (Table.isHivePartitioned(table)) { + if (isHivePartitioned(lowerCaseTableName)) { val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] // unpersist() all RDDs for all Hive-partitions. val unpersistedRDDs = partitionedTable.getAllPartitions.map( @@ -127,9 +140,9 @@ class MemoryMetadataManager { } // Remove MemoryTable's entry from Shark metadata. - _keyToStats.remove(key.toLowerCase) + _keyToStats.remove(lowerCaseTableName) - val tableValue: Option[Table] = _keyToMemoryTable.remove(key.toLowerCase) + val tableValue: Option[Table] = _keyToTable.remove(lowerCaseTableName) return tableValue.flatMap(unpersistTable(_)) } diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 82dcc4ca..a4308a3c 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -40,20 +40,16 @@ import shark.execution.RDDUtils * too much metadata to track, so it should be okay to have a single MemoryTable. */ -private[shark] abstract class Table(val tableName: String, val cacheMode: CacheType.CacheType) { +private[shark] abstract class Table( + var tableName: String, + var cacheMode: CacheType.CacheType) { def getStorageLevel: StorageLevel } -object Table { - - def isHivePartitioned(table: Table) = table.isInstanceOf[PartitionedMemoryTable] - -} - private[shark] class MemoryTable( - val tableName: String, - val cacheMode: CacheType.CacheType) + tableName: String, + cacheMode: CacheType.CacheType) extends Table(tableName, cacheMode) { // RDD that contains the contents of this table. @@ -64,8 +60,8 @@ class MemoryTable( private[shark] class PartitionedMemoryTable( - val tableName: String, - val cacheMode: CacheType.CacheType) + tableName: String, + cacheMode: CacheType.CacheType) extends Table(tableName, cacheMode) { // A map from the Hive-partition key to the RDD that contains contents of that partition. @@ -77,6 +73,8 @@ class PartitionedMemoryTable( // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. private var _partitionCachePolicy: CachePolicy[String, RDD[_]] = _ + private var _partitionCachePolicyName: String = "None" + def getPartition(partitionKey: String): Option[RDD[_]] = { val rddFound = _keyToPartitions.get(partitionKey) if (rddFound.isDefined) _partitionCachePolicy.notifyGet(partitionKey) @@ -94,12 +92,22 @@ class PartitionedMemoryTable( return rddRemoved } - def partitionCachePolicy_= (value: String) { + def setPartitionCachePolicy(cachePolicyStr: String, maxSize: Long) { _partitionCachePolicy = - Class.forName(value).newInstance.asInstanceOf[CachePolicy[String, RDD[_]]] + Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDD[_]]] + val loadFunc: String => RDD[_] = + (partitionKey: String) => { + val partitionRDD = _keyToPartitions.get(partitionKey).get + partitionRDD.persist(RDDUtils.getStorageLevelOfCachedRDDs(getAllPartitions)) + partitionRDD + } + val evictionFunc: (String, RDD[_]) => Unit = + (partitionKey: String, partition: RDD[_]) => partition.unpersist() + _partitionCachePolicy.initialize(maxSize, loadFunc, evictionFunc) + _partitionCachePolicyName = cachePolicyStr } - def partitionCachePolicy: CachePolicy[String, RDD[_]] = _partitionCachePolicy + def getPartitionCachePolicy: String = _partitionCachePolicyName def getAllPartitions = _keyToPartitions.values.toSeq @@ -107,3 +115,4 @@ class PartitionedMemoryTable( def getStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfCachedRDDs(getAllPartitions) } + diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index f3d2d4b1..79f8d94f 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -43,7 +43,7 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) def analyzeAlterTableAddParts(ast: ASTNode) { val tableName = getTableName(ast) // Create a SparkDDLTask only if the table is cached. - if (SharkEnv.memoryMetadataManager.contains(tableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(tableName)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks // and DDLWorks that contain AddPartitionDesc objects. for (ddlTask <- rootTasks) { @@ -57,7 +57,7 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) def alterTableDropParts(ast: ASTNode) { val tableName = getTableName(ast) // Create a SparkDDLTask only if the table is cached. - if (SharkEnv.memoryMetadataManager.contains(tableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(tableName)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks // and DDLWorks that contain AddPartitionDesc objects. for (ddlTask <- rootTasks) { @@ -70,7 +70,7 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) private def analyzeAlterTableRename(astNode: ASTNode) { val oldTableName = getTableName(astNode) - if (SharkEnv.memoryMetadataManager.contains(oldTableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(oldTableName)) { val newTableName = BaseSemanticAnalyzer.getUnescapedName( astNode.getChild(1).asInstanceOf[ASTNode]) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 759b2c7f..a4cef558 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -164,11 +164,12 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } else { // Otherwise, check if we are inserting into a table that was cached. val cachedTableName = tableName.split('.')(1) // Ignore the database name - if (SharkEnv.memoryMetadataManager.contains(cachedTableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(cachedTableName)) { if (hiveSinkOps.size == 1) { // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) - val cacheMode = SharkEnv.memoryMetadataManager.getCacheMode(cachedTableName) + val table = SharkEnv.memoryMetadataManager.getTable(cachedTableName).get + val cacheMode = table.cacheMode var hivePartitionKey = new String if (SharkEnv.memoryMetadataManager.isHivePartitioned(cachedTableName)) { if (cacheMode == CacheType.TACHYON) { @@ -177,7 +178,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey(qb) } - val storageLevel = SharkEnv.memoryMetadataManager.getStorageLevel(cachedTableName) + val storageLevel = table.getStorageLevel OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, cachedTableName, From 58acbcec9e296cf6f5d07c2a4c6a7622a841a808 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 03:47:36 -0700 Subject: [PATCH 091/331] Update tests to account for new MemoryMetadataManager API --- src/main/scala/shark/SharkConfVars.scala | 7 +++ .../scala/shark/execution/SparkDDLTask.scala | 8 +-- .../scala/shark/memstore2/CachePolicy.scala | 2 +- .../memstore2/MemoryMetadataManager.scala | 2 +- .../scala/shark/memstore2/MemoryTable.scala | 2 + src/test/scala/shark/SQLSuite.scala | 50 +++++++++++-------- 6 files changed, 45 insertions(+), 26 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index da2112d7..ab5f0781 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -43,6 +43,13 @@ object SharkConfVars { // Default storage level for cached tables. val STORAGE_LEVEL = new ConfVar("shark.cache.storageLevel", "MEMORY_AND_DISK") + // Class name of the default cache policy for cached, Hive-partitioned tables. + val CACHE_POLICY = new ConfVar( + "shark.cache.partition.cachePolicy.class", "shark.memstore2.LRUCachePolicy") + + // Max size of the cache for the "shark.cache.partition.cachePolicy". + val MAX_CACHE_SIZE = new ConfVar("shark.cache.partition.cachePolicy.maxSize", "10") + // If true, then cache any table whose name ends in "_cached". val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 5cd8ec6a..04fc1028 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.plan.api.StageType -import shark.{LogHelper, SharkEnv} +import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.memstore2.{CacheType, MemoryMetadataManager} @@ -71,8 +71,10 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab val isHivePartitioned = (createTblDesc.getPartCols.size > 0) if (isHivePartitioned) { val tblProps = createTblDesc.getTblProps - val cachePolicyStr = tblProps.get("shark.cache.partition.cachePolicy") - val maxCacheSize = tblProps.get("shark.cache.partition.cachePolicy.maxSize").toLong + val cachePolicyStr = tblProps.getOrElse("shark.cache.partition.cachePolicy.class", + SharkConfVars.CACHE_POLICY.defaultVal) + val maxCacheSize = tblProps.getOrElse("shark.cache.partition.cachePolicy.maxSize", + SharkConfVars.MAX_CACHE_SIZE.defaultVal).toLong SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( tableName, cacheMode, cachePolicyStr, maxCacheSize) } else { diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index d9cc52d4..bdc778e7 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -95,5 +95,5 @@ private[shark] class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolic cache.get(key) } - override def getKeysOfCachedEntries: Seq[K] = cache.asMap.keySet.toSeq + override def getKeysOfCachedEntries: Seq[K] = cache.asMap.keySet.toSeq } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 00776351..6de5e1ac 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -48,7 +48,7 @@ class MemoryMetadataManager { _keyToTable.put(tableName.toLowerCase, newTable) return newTable } - + def createPartitionedMemoryTable( tableName: String, cacheMode: CacheType.CacheType, diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index a4308a3c..27695c59 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -75,6 +75,8 @@ class PartitionedMemoryTable( private var _partitionCachePolicyName: String = "None" + def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) + def getPartition(partitionKey: String): Option[RDD[_]] = { val rddFound = _keyToPartitions.get(partitionKey) if (rddFound.isDefined) _partitionCachePolicy.notifyGet(partitionKey) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index c88bb3d8..aa25f634 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -225,7 +225,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("Use regular CREATE TABLE and '_cached' suffix to create cached table") { sc.runSql("drop table if exists empty_table_cached") sc.runSql("create table empty_table_cached(key string, value string)") - assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable("empty_table_cached")) assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached")) } @@ -233,7 +233,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists empty_table_cached_tbl_props") sc.runSql("""create table empty_table_cached_tbl_props(key string, value string) TBLPROPERTIES('shark.cache' = 'true')""") - assert(SharkEnv.memoryMetadataManager.contains("empty_table_cached_tbl_props")) + assert(SharkEnv.memoryMetadataManager.containsTable("empty_table_cached_tbl_props")) assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached_tbl_props")) } @@ -249,8 +249,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists test_rename") sc.runSql("create table test_oldname_cached as select * from test") sc.runSql("alter table test_oldname_cached rename to test_rename") - assert(!SharkEnv.memoryMetadataManager.contains("test_oldname_cached")) - assert(SharkEnv.memoryMetadataManager.contains("test_rename")) + assert(!SharkEnv.memoryMetadataManager.containsTable("test_oldname_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable("test_rename")) expectSql("select count(*) from test_rename", "500") } @@ -286,7 +286,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists ctas_tbl_props") sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='true') as select * from test""") - assert(SharkEnv.memoryMetadataManager.contains("ctas_tbl_props")) + assert(SharkEnv.memoryMetadataManager.containsTable("ctas_tbl_props")) expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") } @@ -296,7 +296,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { CREATE TABLE ctas_tbl_props_result_should_not_be_cached TBLPROPERTIES ('shark.cache'='false') AS select * from test""") - assert(!SharkEnv.memoryMetadataManager.contains("ctas_tbl_props_should_not_be_cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable("ctas_tbl_props_should_not_be_cached")) } test("cached tables with complex types") { @@ -320,7 +320,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(sc.sql("select d from test_complex_types_cached where a = 'a0'").head === """{"d01":["d011","d012"],"d02":["d021","d022"]}""") - assert(SharkEnv.memoryMetadataManager.contains("test_complex_types_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable("test_complex_types_cached")) } test("disable caching by default") { @@ -328,7 +328,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists should_not_be_cached") sc.runSql("create table should_not_be_cached as select * from test") expectSql("select key from should_not_be_cached where key = 407", "407") - assert(!SharkEnv.memoryMetadataManager.contains("should_not_be_cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable("should_not_be_cached")) sc.runSql("set shark.cache.flag.checkTableName=true") } @@ -337,7 +337,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as select * from test""") expectSql("select val from sharktest5Cached where key = 407", "val_407") - assert(SharkEnv.memoryMetadataManager.contains("sharkTest5Cached")) + assert(SharkEnv.memoryMetadataManager.containsTable("sharkTest5Cached")) } test("dropping cached tables should clean up RDDs") { @@ -345,7 +345,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as select * from test""") sc.runSql("drop table sharkTest5Cached") - assert(!SharkEnv.memoryMetadataManager.contains("sharkTest5Cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable("sharkTest5Cached")) } ////////////////////////////////////////////////////////////////////////////// @@ -356,7 +356,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists empty_part_table_cached") sc.runSql("""create table empty_part_table_cached(key int, value string) partitioned by (keypart int)""") - assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable("empty_part_table_cached")) assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached")) } @@ -364,7 +364,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists empty_part_table_cached_tbl_props") sc.runSql("""create table empty_part_table_cached_tbl_props(key int, value string) partitioned by (keypart int) TBLPROPERTIES('shark.cache' = 'true')""") - assert(SharkEnv.memoryMetadataManager.contains("empty_part_table_cached_tbl_props")) + assert(SharkEnv.memoryMetadataManager.containsTable("empty_part_table_cached_tbl_props")) assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached_tbl_props")) } @@ -375,7 +375,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""alter table alter_part_cached add partition(keypart = 1)""") val tableName = "alter_part_cached" val partitionColumn = "keypart=1" - assert(SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + assert(partitionedTable.containsPartition(partitionColumn)) } // TODO(harvey): Create hadoop file for this. @@ -393,9 +395,11 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""alter table alter_drop_part_cached add partition(keypart = 1)""") val tableName = "alter_drop_part_cached" val partitionColumn = "keypart=1" - assert(SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + assert(partitionedTable.containsPartition(partitionColumn)) sc.runSql("""alter table alter_drop_part_cached drop partition(keypart = 1)""") - assert(!SharkEnv.memoryMetadataManager.containsHivePartition(tableName, partitionColumn)) + assert(!partitionedTable.containsPartition(partitionColumn)) } test("insert into a partition of a cached table") { @@ -454,11 +458,13 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("insert into table drop_mult_part_cached partition(keypart = 5) select * from test") sc.runSql("insert into table drop_mult_part_cached partition(keypart = 9) select * from test") val tableName = "drop_mult_part_cached" - val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") - val keypart5RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=5") - val keypart9RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=9") + assert(SharkEnv.memoryMetadataManager.containsTable("drop_mult_part_cached")) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val keypart1RDD = partitionedTable.containsPartition("keypart=1") + val keypart5RDD = partitionedTable.containsPartition("keypart=5") + val keypart9RDD = partitionedTable.containsPartition("keypart=9") sc.runSql("drop table drop_mult_part_cached ") - assert(!SharkEnv.memoryMetadataManager.contains("drop_mult_part_cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) // All RDDs should have been unpersisted. //assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) //assert(keypart5RDD.get.getStorageLevel == StorageLevel.NONE) @@ -473,9 +479,11 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") val tableName = "drop_union_part_cached" - val keypart1RDD = SharkEnv.memoryMetadataManager.getHivePartition(tableName, "keypart=1") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val keypart1RDD = partitionedTable.getPartition("keypart=1") sc.runSql("drop table drop_union_part_cached") - assert(!SharkEnv.memoryMetadataManager.contains("drop_union_part_cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) // All RDDs should have been unpersisted. //assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) } From 4be578590d878678a450d141c7bdf5b8aff1f73f Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 04:21:57 -0700 Subject: [PATCH 092/331] Some sanity checks for casting in MemoryMetadataManger. --- .../shark/memstore2/MemoryMetadataManager.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 6de5e1ac..fa704a1b 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -73,11 +73,21 @@ class MemoryMetadataManager { def getTable(tableName: String): Option[Table] = _keyToTable.get(tableName.toLowerCase) def getMemoryTable(tableName: String): Option[MemoryTable] = { - _keyToTable.get(tableName.toLowerCase).asInstanceOf[Option[MemoryTable]] + val tableFound = _keyToTable.get(tableName.toLowerCase) + tableFound.foreach(table => + assert(table.isInstanceOf[MemoryTable], + "getMemoryTable() called for a partitioned table.")) + + tableFound.asInstanceOf[Option[MemoryTable]] } def getPartitionedTable(tableName: String): Option[PartitionedMemoryTable] = { - _keyToTable.get(tableName.toLowerCase).asInstanceOf[Option[PartitionedMemoryTable]] + val tableFound = _keyToTable.get(tableName.toLowerCase) + tableFound.foreach(table => + assert(table.isInstanceOf[PartitionedMemoryTable], + "getPartitionedTable() called for a non-partitioned table.")) + + tableFound.asInstanceOf[Option[PartitionedMemoryTable]] } def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { @@ -124,7 +134,7 @@ class MemoryMetadataManager { def unpersistTable(table: Table): Option[RDD[_]] = { var unpersistedRDD: Option[RDD[_]] = None - if (isHivePartitioned(lowerCaseTableName)) { + if (table.isInstanceOf[PartitionedMemoryTable]) { val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] // unpersist() all RDDs for all Hive-partitions. val unpersistedRDDs = partitionedTable.getAllPartitions.map( From 25a79aed39e8bfe900d728d6c6f1e710f4b6e058 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 14:56:57 -0700 Subject: [PATCH 093/331] Test skeleton for cache policies. --- src/test/scala/shark/SQLSuite.scala | 140 ++++++++++++++++++++++++---- 1 file changed, 124 insertions(+), 16 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index aa25f634..2ae2d107 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -20,6 +20,8 @@ package shark import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.apache.spark.storage.StorageLevel + import shark.api.QueryExecutionException @@ -363,7 +365,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { sc.runSql("drop table if exists empty_part_table_cached_tbl_props") sc.runSql("""create table empty_part_table_cached_tbl_props(key int, value string) - partitioned by (keypart int) TBLPROPERTIES('shark.cache' = 'true')""") + partitioned by (keypart int) tblproperties('shark.cache' = 'true')""") assert(SharkEnv.memoryMetadataManager.containsTable("empty_part_table_cached_tbl_props")) assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached_tbl_props")) } @@ -380,14 +382,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(partitionedTable.containsPartition(partitionColumn)) } - // TODO(harvey): Create hadoop file for this. - // test("alter cached table by adding a new partition, with a provided location") { - // sc.runSql("drop table if exists alter_part_location_cached") - // sc.runSql("""create table alter_part_location_cached(key int, val string) - // partitioned by (keypart int)""") - // sc.runSql("""alter table alter_part_location_cached add partition(keypart = 1)""") - // } - test("alter cached table by dropping a partition") { sc.runSql("drop table if exists alter_drop_part_cached") sc.runSql("""create table alter_drop_part_cached(key int, value string) @@ -460,15 +454,15 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val tableName = "drop_mult_part_cached" assert(SharkEnv.memoryMetadataManager.containsTable("drop_mult_part_cached")) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val keypart1RDD = partitionedTable.containsPartition("keypart=1") - val keypart5RDD = partitionedTable.containsPartition("keypart=5") - val keypart9RDD = partitionedTable.containsPartition("keypart=9") + val keypart1RDD = partitionedTable.getPartition("keypart=1") + val keypart5RDD = partitionedTable.getPartition("keypart=5") + val keypart9RDD = partitionedTable.getPartition("keypart=9") sc.runSql("drop table drop_mult_part_cached ") assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) // All RDDs should have been unpersisted. - //assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) - //assert(keypart5RDD.get.getStorageLevel == StorageLevel.NONE) - //assert(keypart9RDD.get.getStorageLevel == StorageLevel.NONE) + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + assert(keypart5RDD.get.getStorageLevel == StorageLevel.NONE) + assert(keypart9RDD.get.getStorageLevel == StorageLevel.NONE) } test("drop cached partition represented by a UnionRDD (i.e., the result of multiple inserts)") { @@ -485,9 +479,123 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table drop_union_part_cached") assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) // All RDDs should have been unpersisted. - //assert(keypart1RDD.getStorageLevel == StorageLevel.NONE) + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + } + + ////////////////////////////////////////////////////////////////////////////// + // RDD(partition) eviction policy for cached Hive-partititioned tables + ////////////////////////////////////////////////////////////////////////////// + test("LRU: RDDs are evicted when the max size is reached.") { + sc.runSql("drop table if exists evict_partitions_maxSize_cached") + sc.runSql(""" + create table evict_partitions_maxSize_cached(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', + 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', + 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') + """) + sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 1) select * from test") + sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 2) select * from test") + sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 3) select * from test") + val tableName = "evict_partitions_maxSize_cached" + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val keypart1RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 4) select * from test") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for get()s.") { + sc.runSql("drop table if exists evict_partitions_get_ordering_cached") + sc.runSql(""" + create table evict_partitions_get_order_cached(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', + 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', + 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') + """) + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 2) + select * from test""") + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 3) + select * from test""") + val tableName = "evict_partitions_get_order_cached" + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val keypart1RDD = partitionedTable.getPartition("keypart=1") + val keypart2RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("select count(1) from evict_partitions_get_order_cached where keypart = 1") + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 4) + select * from test""") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) } + test("LRU: RDD eviction accounts for put()s.") { + sc.runSql("drop table if exists evict_partitions_get_ordering_cached") + sc.runSql(""" + create table evict_partitions_get_order_cached(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', + 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', + 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') + """) + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 2) + select * from test""") + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 3) + select * from test""") + val tableName = "evict_partitions_get_order_cached" + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val keypart1RDD = partitionedTable.getPartition("keypart=1") + val keypart2RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 4) + select * from test""") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + } + + test("LRU: get() reloads an RDD previously unpersist()'d.") { + sc.runSql("drop table if exists reload_evicted_partition_cached") + sc.runSql(""" + create table reload_evicted_partition_cached(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', + 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', + 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') + """) + sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 2) + select * from test""") + sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 3) + select * from test""") + val tableName = "reload_evicted_partition_cached" + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val keypart1RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 4) + select * from test""") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + + // Scanning (keypart = 1) should reload it into the cache, and (keypart = 2) + // should have been evicted + sc.runSql("select count(1) from reload_evicted_partition_cached where keypart = 1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val keypart2RDD = partitionedTable.getPartition("keypart=1") + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + } ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// From 02ce5dde06f0550b2cda09047707bcc105b245d8 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 16:18:21 -0700 Subject: [PATCH 094/331] Add API for accessing cache stats. --- .../scala/shark/memstore2/CachePolicy.scala | 55 +++++++++++++++---- 1 file changed, 44 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index bdc778e7..80692633 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -18,11 +18,10 @@ package shark.memstore2 import com.google.common.cache._ - import scala.collection.JavaConversions._ -private[shark] abstract class CachePolicy[K, V] { +trait CachePolicy[K, V] { protected var maxSize: Long = _ @@ -49,21 +48,40 @@ private[shark] abstract class CachePolicy[K, V] { def getKeysOfCachedEntries: Seq[K] def getMaxSize = maxSize + + def getHitRate: Option[Double] = None + + def getEvictionCount: Option[Long] = None } -private[shark] class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { +class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { var isInitialized = false + var hasRecordedStats = false var cache: LoadingCache[K, V] = _ + var cacheStats: Option[CacheStats] = None override def initialize( - maxSize: Long, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit + maxSize: Long, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit + ): Unit = { + initialize(maxSize, loadFunc, evictionFunc, false) + } + + def initialize( + maxSize: Long, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit, + shouldRecordStats: Boolean ): Unit = { super.initialize(maxSize, loadFunc, evictionFunc) var builder = CacheBuilder.newBuilder().maximumSize(maxSize) + if (shouldRecordStats) { + builder.recordStats() + hasRecordedStats = true + } val removalListener = new RemovalListener[K, V] { @@ -76,24 +94,39 @@ private[shark] class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolic def load(key: K): V = loadFunc(key) } - cache = builder.removalListener(removalListener).build(cacheLoader) + cache = builder + .removalListener(removalListener) + .build(cacheLoader) isInitialized = true } override def notifyPut(key: K, value: V): Unit = { - assert(isInitialized, "LRUCachePolicy must be initialize()'d.") + assert(isInitialized, "Must initialize() LRUCachePolicy.") cache.put(key, value) } override def notifyRemove(key: K, value: V): Unit = { - assert(isInitialized, "LRUCachePolicy must be initialize()'d.") + assert(isInitialized, "Must initialize() LRUCachePolicy.") cache.invalidate(key, value) } override def notifyGet(key: K): Unit = { - assert(isInitialized, "LRUCachePolicy must be initialize()'d.") + assert(isInitialized, "Must initialize() LRUCachePolicy.") cache.get(key) } - override def getKeysOfCachedEntries: Seq[K] = cache.asMap.keySet.toSeq + override def getKeysOfCachedEntries: Seq[K] = { + assert(isInitialized, "Must initialize() LRUCachePolicy.") + return cache.asMap.keySet.toSeq + } + + override def getHitRate(): Option[Double] = { + val hitRate = if (hasRecordedStats) Some(cache.stats.hitRate) else None + return hitRate + } + + override def getEvictionCount(): Option[Long] = { + val evictionCount = if (hasRecordedStats) Some(cache.stats.evictionCount) else None + return evictionCount + } } From 14709651303bc33613fb1264bf9aed42329f312d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 18:02:46 -0700 Subject: [PATCH 095/331] Add table property for whether or not to record cache stats. --- src/main/scala/shark/SharkConfVars.scala | 6 +++++- .../scala/shark/execution/SparkDDLTask.scala | 4 +++- .../scala/shark/memstore2/CachePolicy.scala | 13 +++---------- .../shark/memstore2/MemoryMetadataManager.scala | 5 +++-- .../scala/shark/memstore2/MemoryTable.scala | 17 +++++++++-------- src/test/scala/shark/SQLSuite.scala | 17 +++++++++++------ 6 files changed, 34 insertions(+), 28 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index ab5f0781..c64f1acc 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -47,9 +47,13 @@ object SharkConfVars { val CACHE_POLICY = new ConfVar( "shark.cache.partition.cachePolicy.class", "shark.memstore2.LRUCachePolicy") - // Max size of the cache for the "shark.cache.partition.cachePolicy". + // Max size of the cache specified by the "shark.cache.partition.cachePolicy" property. val MAX_CACHE_SIZE = new ConfVar("shark.cache.partition.cachePolicy.maxSize", "10") + // Whether the cache specified by "shark.cache.partition.cachePolicy" should record statistics. + val SHOULD_RECORD_CACHE_STATS = new ConfVar( + "shark.cache.partition.cachePolicy.shouldRecordStats", false) + // If true, then cache any table whose name ends in "_cached". val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 04fc1028..02212ac8 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -75,8 +75,10 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab SharkConfVars.CACHE_POLICY.defaultVal) val maxCacheSize = tblProps.getOrElse("shark.cache.partition.cachePolicy.maxSize", SharkConfVars.MAX_CACHE_SIZE.defaultVal).toLong + val shouldRecordStats = tblProps.getOrElse("shark.cache.partition.shouldRecordStats", + SharkConfVars.SHOULD_RECORD_CACHE_STATS.defaultBoolVal.toString).toBoolean SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( - tableName, cacheMode, cachePolicyStr, maxCacheSize) + tableName, cacheMode, cachePolicyStr, maxCacheSize, shouldRecordStats) } else { val newTable = SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode) newTable.tableRDD = new EmptyRDD(SharkEnv.sc) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index 80692633..1a74ed7c 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -32,7 +32,8 @@ trait CachePolicy[K, V] { def initialize( maxSize: Long, loadFunc: (K => V), - evictionFunc: (K, V) => Unit + evictionFunc: (K, V) => Unit, + shouldRecordStats: Boolean ): Unit = { this.maxSize = maxSize this.loadFunc = loadFunc @@ -62,20 +63,12 @@ class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { var cacheStats: Option[CacheStats] = None override def initialize( - maxSize: Long, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit - ): Unit = { - initialize(maxSize, loadFunc, evictionFunc, false) - } - - def initialize( maxSize: Long, loadFunc: (K => V), evictionFunc: (K, V) => Unit, shouldRecordStats: Boolean ): Unit = { - super.initialize(maxSize, loadFunc, evictionFunc) + super.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) var builder = CacheBuilder.newBuilder().maximumSize(maxSize) if (shouldRecordStats) { diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index fa704a1b..cbeb4c78 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -53,10 +53,11 @@ class MemoryMetadataManager { tableName: String, cacheMode: CacheType.CacheType, cachePolicyStr: String, - cachePolicyMaxSize: Long + cachePolicyMaxSize: Long, + shouldRecordStats: Boolean ): PartitionedMemoryTable = { var newTable = new PartitionedMemoryTable(tableName.toLowerCase, cacheMode) - newTable.setPartitionCachePolicy(cachePolicyStr, cachePolicyMaxSize) + newTable.setPartitionCachePolicy(cachePolicyStr, cachePolicyMaxSize, shouldRecordStats) _keyToTable.put(tableName.toLowerCase, newTable) return newTable } diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 27695c59..8811a810 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -34,12 +34,7 @@ import shark.execution.RDDUtils * Note that a Hive-partition of a table is different from an RDD partition. Each Hive-partition * is stored as a subdirectory of the table subdirectory in the warehouse directory * (e.g. /user/hive/warehouse). So, every Hive-Partition is loaded into Shark as an RDD. - * - * TODO(harvey): It could be useful to make MemoryTable a parent class, and have other table types, - * such as HivePartitionedTable or TachyonTable, subclass it. For now, there isn't - * too much metadata to track, so it should be okay to have a single MemoryTable. */ - private[shark] abstract class Table( var tableName: String, var cacheMode: CacheType.CacheType) { @@ -94,7 +89,11 @@ class PartitionedMemoryTable( return rddRemoved } - def setPartitionCachePolicy(cachePolicyStr: String, maxSize: Long) { + def setPartitionCachePolicy( + cachePolicyStr: String, + maxSize: Long, + shouldRecordStats: Boolean + ) { _partitionCachePolicy = Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDD[_]]] val loadFunc: String => RDD[_] = @@ -105,11 +104,13 @@ class PartitionedMemoryTable( } val evictionFunc: (String, RDD[_]) => Unit = (partitionKey: String, partition: RDD[_]) => partition.unpersist() - _partitionCachePolicy.initialize(maxSize, loadFunc, evictionFunc) + _partitionCachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) _partitionCachePolicyName = cachePolicyStr } - def getPartitionCachePolicy: String = _partitionCachePolicyName + def partitionCachePolicyName: String = _partitionCachePolicyName + + def partitionCachePolicy: CachePolicy[String, RDD[_]] = _partitionCachePolicy def getAllPartitions = _keyToPartitions.values.toSeq diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 2ae2d107..5bae4b30 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -494,15 +494,19 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') """) - sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 1) select * from test") - sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 2) select * from test") - sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 3) select * from test") + sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 2) + select * from test""") + sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 3) + select * from test""") val tableName = "evict_partitions_maxSize_cached" assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get val keypart1RDD = partitionedTable.getPartition("keypart=1") assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - sc.runSql("insert into table evict_partitions_maxSize_cached partition(keypart = 4) select * from test") + sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 4) + select * from test""") assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) } @@ -589,13 +593,14 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { select * from test""") assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) - // Scanning (keypart = 1) should reload it into the cache, and (keypart = 2) - // should have been evicted + // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and + // cause eviction of the RDD for partition (keypart = 2). sc.runSql("select count(1) from reload_evicted_partition_cached where keypart = 1") assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) val keypart2RDD = partitionedTable.getPartition("keypart=1") assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) } + ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// From 4a747748102308cf9d79ae0280dd11f5c0cddb83 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 18:20:40 -0700 Subject: [PATCH 096/331] Add unit test for 'shark.cache.partition.CachePolicy.shouldRecordStats'. --- .../scala/shark/memstore2/MemoryTable.scala | 20 ++++++++-------- src/test/scala/shark/SQLSuite.scala | 23 +++++++++++++++++++ 2 files changed, 33 insertions(+), 10 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 8811a810..8b1392d6 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -66,26 +66,26 @@ class PartitionedMemoryTable( // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. - private var _partitionCachePolicy: CachePolicy[String, RDD[_]] = _ + private var _cachePolicy: CachePolicy[String, RDD[_]] = _ - private var _partitionCachePolicyName: String = "None" + private var _cachePolicyName: String = "None" def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) def getPartition(partitionKey: String): Option[RDD[_]] = { val rddFound = _keyToPartitions.get(partitionKey) - if (rddFound.isDefined) _partitionCachePolicy.notifyGet(partitionKey) + if (rddFound.isDefined) _cachePolicy.notifyGet(partitionKey) return rddFound } def putPartition(partitionKey: String, rdd: RDD[_]): Option[RDD[_]] = { - _partitionCachePolicy.notifyPut(partitionKey, rdd) + _cachePolicy.notifyPut(partitionKey, rdd) _keyToPartitions.put(partitionKey, rdd) } def removePartition(partitionKey: String): Option[RDD[_]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) - if (rddRemoved.isDefined) _partitionCachePolicy.notifyRemove(partitionKey, rddRemoved.get) + if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey, rddRemoved.get) return rddRemoved } @@ -94,7 +94,7 @@ class PartitionedMemoryTable( maxSize: Long, shouldRecordStats: Boolean ) { - _partitionCachePolicy = + _cachePolicy = Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDD[_]]] val loadFunc: String => RDD[_] = (partitionKey: String) => { @@ -104,13 +104,13 @@ class PartitionedMemoryTable( } val evictionFunc: (String, RDD[_]) => Unit = (partitionKey: String, partition: RDD[_]) => partition.unpersist() - _partitionCachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) - _partitionCachePolicyName = cachePolicyStr + _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) + _cachePolicyName = cachePolicyStr } - def partitionCachePolicyName: String = _partitionCachePolicyName + def cachePolicyName: String = _cachePolicyName - def partitionCachePolicy: CachePolicy[String, RDD[_]] = _partitionCachePolicy + def cachePolicy: CachePolicy[String, RDD[_]] = _cachePolicy def getAllPartitions = _keyToPartitions.values.toSeq diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 5bae4b30..40535f17 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -601,6 +601,29 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) } + test("LRU: user is able to toggle cache stats recording") { + sc.runSql("drop table if exists record_partition_cache_stats") + sc.runSql(""" + create table record_partition_cache_stats(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache' = 'true', + 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', + 'shark.cache.partition.cachePolicy.shouldRecordStats' = 'true') + """) + val tableName = "record_partition_cache_stats" + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + sc.runSql("""insert into table record_partition_cache_stats partition(keypart = 1) + select * from test""") + val lruCachePolicy = partitionedTable.cachePolicy + val hitRate = lruCachePolicy.getHitRate + assert(hitRate.isDefined) + assert(hitRate.get == 1.0) + val evictionCount = lruCachePolicy.getEvictionCount + assert(evictionCount.isDefined) + assert(evictionCount.get == 0) + } + ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// From 5888e0becbc89396bd20af01e324f5d8ca8a45ab Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 19:16:37 -0700 Subject: [PATCH 097/331] Move CachePolicy unit tests from SQLSuite to a new CachePolicySuite. --- src/test/scala/shark/CachePolicySuite.scala | 185 ++++++++++++++++++++ src/test/scala/shark/SQLSuite.scala | 138 --------------- 2 files changed, 185 insertions(+), 138 deletions(-) create mode 100644 src/test/scala/shark/CachePolicySuite.scala diff --git a/src/test/scala/shark/CachePolicySuite.scala b/src/test/scala/shark/CachePolicySuite.scala new file mode 100644 index 00000000..57f700c0 --- /dev/null +++ b/src/test/scala/shark/CachePolicySuite.scala @@ -0,0 +1,185 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.storage.StorageLevel + +import shark.api.QueryExecutionException +import shark.memstore2.PartitionedMemoryTable + + +class CachePolicySuite extends FunSuite with BeforeAndAfterAll { + + val WAREHOUSE_PATH = TestUtils.getWarehousePath() + val METASTORE_PATH = TestUtils.getMetastorePath() + val MASTER = "local" + + var sc: SharkContext = _ + + override def beforeAll() { + sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) + sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + + METASTORE_PATH + ";create=true") + sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) + + // test + sc.runSql("drop table if exists test") + sc.runSql("CREATE TABLE test (key INT, val STRING)") + sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test") + sc.runSql("drop table if exists test_cached") + sc.runSql("CREATE TABLE test_cached AS SELECT * FROM test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + private def createCachedPartitionedTable( + tableName: String, + numPartitionsToCreate: Int, + cachePolicyClassName: String, + shouldRecordStats: Boolean = false + ): PartitionedMemoryTable = { + sc.runSql("drop table if exists %s".format(tableName)) + sc.runSql(""" + create table %s(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache' = 'true', + 'shark.cache.partition.cachePolicy.maxSize' = '3', + 'shark.cache.partition.cachePolicy.class' = '%s', + 'shark.cache.storageLevel' = 'MEMORY_AND_DISK', + 'shark.cache.partition.cachePolicy.shouldRecordStats' = '%b') + """.format( + tableName, + cachePolicyClassName, + shouldRecordStats)) + for (partitionNum <- 0 until numPartitionsToCreate) { + sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = %d) + select * from test""".format(partitionNum)) + } + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + return partitionedTable + } + + test("LRU: RDDs are evicted when the max size is reached.") { + val tableName = "evict_partitions_maxSize" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table evict_partitions_maxSize partition(keypart = 4) + select * from test""") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for get()s.") { + val tableName = "evict_partitions_with_get" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.getPartition("keypart=1") + val keypart2RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("select count(1) from evict_partitions_with_get where keypart = 1") + sc.runSql("""insert into table evict_partitions_with_get partition(keypart = 4) + select * from test""") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for put()s.") { + val tableName = "evict_partitions_with_put" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val keypart1RDD = partitionedTable.getPartition("keypart=1") + val keypart2RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + select * from test""") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + } + + test("LRU: get() reloads an RDD previously unpersist()'d.") { + val tableName = "reload_evicted_partition" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val keypart1RDD = partitionedTable.getPartition("keypart=1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) + select * from test""") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + + // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and + // cause eviction of the RDD for partition (keypart = 2). + sc.runSql("select count(1) from reload_evicted_partition where keypart = 1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val keypart2RDD = partitionedTable.getPartition("keypart=1") + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + } + + test("LRU: cache stats are not recorded by default") { + val tableName = "dont_record_partition_cache_stats" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */, + "shark.memstore2.LRUCachePolicy", + true /* shouldRecordStats */) + val lruCachePolicy = partitionedTable.cachePolicy + val hitRate = lruCachePolicy.getHitRate + assert(hitRate.isEmpty) + val evictionCount = lruCachePolicy.getEvictionCount + assert(evictionCount.isEmpty) + } + + test("LRU: record cache stats if user specifies it") { + val tableName = "record_partition_cache_stats" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */, + "shark.memstore2.LRUCachePolicy", + true /* shouldRecordStats */) + val lruCachePolicy = partitionedTable.cachePolicy + val hitRate = lruCachePolicy.getHitRate + assert(hitRate.isDefined) + assert(hitRate.get == 1.0) + val evictionCount = lruCachePolicy.getEvictionCount + assert(evictionCount.isDefined) + assert(evictionCount.get == 0) + } +} diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 40535f17..35610340 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -485,144 +485,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // RDD(partition) eviction policy for cached Hive-partititioned tables ////////////////////////////////////////////////////////////////////////////// - test("LRU: RDDs are evicted when the max size is reached.") { - sc.runSql("drop table if exists evict_partitions_maxSize_cached") - sc.runSql(""" - create table evict_partitions_maxSize_cached(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', - 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', - 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') - """) - sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 2) - select * from test""") - sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 3) - select * from test""") - val tableName = "evict_partitions_maxSize_cached" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val keypart1RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = 4) - select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for get()s.") { - sc.runSql("drop table if exists evict_partitions_get_ordering_cached") - sc.runSql(""" - create table evict_partitions_get_order_cached(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', - 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', - 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') - """) - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 2) - select * from test""") - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 3) - select * from test""") - val tableName = "evict_partitions_get_order_cached" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val keypart1RDD = partitionedTable.getPartition("keypart=1") - val keypart2RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - sc.runSql("select count(1) from evict_partitions_get_order_cached where keypart = 1") - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 4) - select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for put()s.") { - sc.runSql("drop table if exists evict_partitions_get_ordering_cached") - sc.runSql(""" - create table evict_partitions_get_order_cached(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', - 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', - 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') - """) - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 2) - select * from test""") - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 3) - select * from test""") - val tableName = "evict_partitions_get_order_cached" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val keypart1RDD = partitionedTable.getPartition("keypart=1") - val keypart2RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_get_order_cached partition(keypart = 4) - select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) - } - - test("LRU: get() reloads an RDD previously unpersist()'d.") { - sc.runSql("drop table if exists reload_evicted_partition_cached") - sc.runSql(""" - create table reload_evicted_partition_cached(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache.partition.cachePolicy.maxSize' = '3', - 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', - 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') - """) - sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 2) - select * from test""") - sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 3) - select * from test""") - val tableName = "reload_evicted_partition_cached" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val keypart1RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table reload_evicted_partition_cached partition(keypart = 4) - select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) - - // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and - // cause eviction of the RDD for partition (keypart = 2). - sc.runSql("select count(1) from reload_evicted_partition_cached where keypart = 1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - val keypart2RDD = partitionedTable.getPartition("keypart=1") - assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) - } - - test("LRU: user is able to toggle cache stats recording") { - sc.runSql("drop table if exists record_partition_cache_stats") - sc.runSql(""" - create table record_partition_cache_stats(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache' = 'true', - 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy', - 'shark.cache.partition.cachePolicy.shouldRecordStats' = 'true') - """) - val tableName = "record_partition_cache_stats" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - sc.runSql("""insert into table record_partition_cache_stats partition(keypart = 1) - select * from test""") - val lruCachePolicy = partitionedTable.cachePolicy - val hitRate = lruCachePolicy.getHitRate - assert(hitRate.isDefined) - assert(hitRate.get == 1.0) - val evictionCount = lruCachePolicy.getEvictionCount - assert(evictionCount.isDefined) - assert(evictionCount.get == 0) - } ////////////////////////////////////////////////////////////////////////////// // Tableau bug From 186397e4192e82c884972cd02c595ba7be33e0e2 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 19:42:52 -0700 Subject: [PATCH 098/331] Move RDD unpersist() helper methods to RDDUtils. --- src/main/scala/shark/execution/RDDUtils.scala | 14 ++++++ .../memstore2/MemoryMetadataManager.scala | 50 +++++++------------ .../scala/shark/memstore2/MemoryTable.scala | 2 +- 3 files changed, 33 insertions(+), 33 deletions(-) diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 60ac9eee..4d93227a 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -53,6 +53,20 @@ object RDDUtils { } } + def unpersistRDD(rdd: RDD[_]): RDD[_] = { + rdd match { + case u: UnionRDD[_] => { + // Recursively unpersist() all RDDs that compose the UnionRDD. + u.unpersist() + u.rdds.map { + r => r.unpersist() + } + } + case r => r.unpersist() + } + return rdd + } + /** * Repartition an RDD using the given partitioner. This is similar to Spark's partitionBy, * except we use the Shark shuffle serializer. diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index cbeb4c78..bbd0f53f 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -133,48 +133,34 @@ class MemoryMetadataManager { def unpersist(tableName: String): Option[RDD[_]] = { val lowerCaseTableName = tableName.toLowerCase - def unpersistTable(table: Table): Option[RDD[_]] = { - var unpersistedRDD: Option[RDD[_]] = None - if (table.isInstanceOf[PartitionedMemoryTable]) { - val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] - // unpersist() all RDDs for all Hive-partitions. - val unpersistedRDDs = partitionedTable.getAllPartitions.map( - rdd => unpersistRDD(rdd)).asInstanceOf[Seq[RDD[Any]]] - if (unpersistedRDDs.size > 0) { - val unionedRDD = new UnionRDD(unpersistedRDDs.head.context, unpersistedRDDs) - unpersistedRDD = Some(unionedRDD) - } - } else { - unpersistedRDD = Some(unpersistRDD(table.asInstanceOf[MemoryTable].tableRDD)) - } - return unpersistedRDD - } - // Remove MemoryTable's entry from Shark metadata. _keyToStats.remove(lowerCaseTableName) val tableValue: Option[Table] = _keyToTable.remove(lowerCaseTableName) - return tableValue.flatMap(unpersistTable(_)) - } - - def unpersistRDD(rdd: RDD[_]): RDD[_] = { - rdd match { - case u: UnionRDD[_] => { - // Recursively unpersist() all RDDs that compose the UnionRDD. - u.unpersist() - u.rdds.map { - r => r.unpersist() - } - } - case r => r.unpersist() - } - return rdd + return tableValue.flatMap(MemoryMetadataManager.unpersistTable(_)) } } object MemoryMetadataManager { + def unpersistTable(table: Table): Option[RDD[_]] = { + var unpersistedRDD: Option[RDD[_]] = None + if (table.isInstanceOf[PartitionedMemoryTable]) { + val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] + // unpersist() all RDDs for all Hive-partitions. + val unpersistedRDDs = partitionedTable.getAllPartitions.map( + rdd => RDDUtils.unpersistRDD(rdd)).asInstanceOf[Seq[RDD[Any]]] + if (unpersistedRDDs.size > 0) { + val unionedRDD = new UnionRDD(unpersistedRDDs.head.context, unpersistedRDDs) + unpersistedRDD = Some(unionedRDD) + } + } else { + unpersistedRDD = Some(RDDUtils.unpersistRDD(table.asInstanceOf[MemoryTable].tableRDD)) + } + return unpersistedRDD + } + def makeHivePartitionKeyStr( partitionColumns: Seq[String], partitionColumnToValue: JavaMap[String, String]): String = { diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 8b1392d6..83213137 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -103,7 +103,7 @@ class PartitionedMemoryTable( partitionRDD } val evictionFunc: (String, RDD[_]) => Unit = - (partitionKey: String, partition: RDD[_]) => partition.unpersist() + (partitionKey: String, partition: RDD[_]) => RDDUtils.unpersistRDD(partition) _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) _cachePolicyName = cachePolicyStr } From cfec8ed5b6baeb0a3a1225b337d3379ea24d1ae4 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 19:43:14 -0700 Subject: [PATCH 099/331] Unit test to make sure that LRUCachePolicy is the default. --- src/test/scala/shark/CachePolicySuite.scala | 28 ++++++++++++++++++--- 1 file changed, 24 insertions(+), 4 deletions(-) diff --git a/src/test/scala/shark/CachePolicySuite.scala b/src/test/scala/shark/CachePolicySuite.scala index 57f700c0..d537f400 100644 --- a/src/test/scala/shark/CachePolicySuite.scala +++ b/src/test/scala/shark/CachePolicySuite.scala @@ -57,6 +57,7 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { private def createCachedPartitionedTable( tableName: String, + maxSize: Int, numPartitionsToCreate: Int, cachePolicyClassName: String, shouldRecordStats: Boolean = false @@ -66,27 +67,41 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { create table %s(key int, value string) partitioned by (keypart int) tblproperties('shark.cache' = 'true', - 'shark.cache.partition.cachePolicy.maxSize' = '3', + 'shark.cache.partition.cachePolicy.maxSize' = '%d', 'shark.cache.partition.cachePolicy.class' = '%s', 'shark.cache.storageLevel' = 'MEMORY_AND_DISK', 'shark.cache.partition.cachePolicy.shouldRecordStats' = '%b') """.format( tableName, + maxSize, cachePolicyClassName, shouldRecordStats)) - for (partitionNum <- 0 until numPartitionsToCreate) { - sc.runSql("""insert into table evict_partitions_maxSize_cached partition(keypart = %d) - select * from test""".format(partitionNum)) + var partitionNum = 0 + while (partitionNum < numPartitionsToCreate) { + sc.runSql("""insert into table %s partition(keypart = %d) + select * from test_cached""".format(tableName, partitionNum)) + partitionNum += 1 } assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get return partitionedTable } + test("shark.memstore2.LRUCachePolicy is the default policy") { + val tableName = "lru_default_policy_cached" + sc.runSql("""create table lru_default_policy_cached(key int, value string) + partitioned by (keypart int)""") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val cachePolicy = partitionedTable.cachePolicy + assert(cachePolicy.isInstanceOf[shark.memstore2.LRUCachePolicy[_, _]]) + } + test("LRU: RDDs are evicted when the max size is reached.") { val tableName = "evict_partitions_maxSize" val partitionedTable = createCachedPartitionedTable( tableName, + 3 /* maxSize */, 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") val keypart1RDD = partitionedTable.getPartition("keypart=1") @@ -100,6 +115,7 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { val tableName = "evict_partitions_with_get" val partitionedTable = createCachedPartitionedTable( tableName, + 3 /* maxSize */, 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") val keypart1RDD = partitionedTable.getPartition("keypart=1") @@ -117,6 +133,7 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { val tableName = "evict_partitions_with_put" val partitionedTable = createCachedPartitionedTable( tableName, + 3 /* maxSize */, 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) @@ -136,6 +153,7 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { val tableName = "reload_evicted_partition" val partitionedTable = createCachedPartitionedTable( tableName, + 3 /* maxSize */, 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) @@ -157,6 +175,7 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { val tableName = "dont_record_partition_cache_stats" val partitionedTable = createCachedPartitionedTable( tableName, + 3 /* maxSize */, 1 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy", true /* shouldRecordStats */) @@ -171,6 +190,7 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { val tableName = "record_partition_cache_stats" val partitionedTable = createCachedPartitionedTable( tableName, + 3 /* maxSize */, 1 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy", true /* shouldRecordStats */) From a94e65a639f5936bfadc7f12defacdf90230172c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 20:44:50 -0700 Subject: [PATCH 100/331] Distinguish between the current StorageLevel of a table's RDD(s) and the preferred StorageLevel passed at table creation time. --- .../execution/MemoryStoreSinkOperator.scala | 2 +- .../scala/shark/execution/SparkDDLTask.scala | 14 +++++-- .../memstore2/MemoryMetadataManager.scala | 9 +++-- .../scala/shark/memstore2/MemoryTable.scala | 37 +++++++++++++------ .../shark/parse/SharkSemanticAnalyzer.scala | 8 ++-- 5 files changed, 47 insertions(+), 23 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 5eba8ec3..6bbdc49e 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -153,7 +153,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { } } else { val table = SharkEnv.memoryMetadataManager.getMemoryTable(tableName).getOrElse( - SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode)) + SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode, storageLevel)) rdd.setName(tableName) table.tableRDD = rdd } diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 02212ac8..0f8213b5 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -68,9 +68,11 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { val tableName = createTblDesc.getTableName + val tblProps = createTblDesc.getTblProps + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) val isHivePartitioned = (createTblDesc.getPartCols.size > 0) if (isHivePartitioned) { - val tblProps = createTblDesc.getTblProps val cachePolicyStr = tblProps.getOrElse("shark.cache.partition.cachePolicy.class", SharkConfVars.CACHE_POLICY.defaultVal) val maxCacheSize = tblProps.getOrElse("shark.cache.partition.cachePolicy.maxSize", @@ -78,9 +80,15 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab val shouldRecordStats = tblProps.getOrElse("shark.cache.partition.shouldRecordStats", SharkConfVars.SHOULD_RECORD_CACHE_STATS.defaultBoolVal.toString).toBoolean SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( - tableName, cacheMode, cachePolicyStr, maxCacheSize, shouldRecordStats) + tableName, + cacheMode, + preferredStorageLevel, + cachePolicyStr, + maxCacheSize, + shouldRecordStats) } else { - val newTable = SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode) + val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( + tableName, cacheMode, preferredStorageLevel) newTable.tableRDD = new EmptyRDD(SharkEnv.sc) } } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index bbd0f53f..c1aca720 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -42,9 +42,10 @@ class MemoryMetadataManager { def createMemoryTable( tableName: String, - cacheMode: CacheType.CacheType + cacheMode: CacheType.CacheType, + preferredStorageLevel: StorageLevel ): MemoryTable = { - var newTable = new MemoryTable(tableName.toLowerCase, cacheMode) + var newTable = new MemoryTable(tableName.toLowerCase, cacheMode, preferredStorageLevel) _keyToTable.put(tableName.toLowerCase, newTable) return newTable } @@ -52,11 +53,13 @@ class MemoryMetadataManager { def createPartitionedMemoryTable( tableName: String, cacheMode: CacheType.CacheType, + preferredStorageLevel: StorageLevel, cachePolicyStr: String, cachePolicyMaxSize: Long, shouldRecordStats: Boolean ): PartitionedMemoryTable = { - var newTable = new PartitionedMemoryTable(tableName.toLowerCase, cacheMode) + var newTable = new PartitionedMemoryTable( + tableName.toLowerCase, cacheMode, preferredStorageLevel) newTable.setPartitionCachePolicy(cachePolicyStr, cachePolicyMaxSize, shouldRecordStats) _keyToTable.put(tableName.toLowerCase, newTable) return newTable diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 83213137..cd3279ea 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -22,7 +22,7 @@ import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ConcurrentMap -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{EmptyRDD, RDD} import org.apache.spark.storage.StorageLevel import shark.execution.RDDUtils @@ -37,27 +37,35 @@ import shark.execution.RDDUtils */ private[shark] abstract class Table( var tableName: String, - var cacheMode: CacheType.CacheType) { - def getStorageLevel: StorageLevel + var cacheMode: CacheType.CacheType, + var preferredStorageLevel: StorageLevel) { + + def getPreferredStorageLevel: StorageLevel + + def getCurrentStorageLevel: StorageLevel } private[shark] class MemoryTable( tableName: String, - cacheMode: CacheType.CacheType) - extends Table(tableName, cacheMode) { + cacheMode: CacheType.CacheType, + preferredStorageLevel: StorageLevel) + extends Table(tableName, cacheMode, preferredStorageLevel) { // RDD that contains the contents of this table. var tableRDD: RDD[_] = _ - override def getStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfCachedRDD(tableRDD) + override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel + + override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfCachedRDD(tableRDD) } private[shark] class PartitionedMemoryTable( tableName: String, - cacheMode: CacheType.CacheType) - extends Table(tableName, cacheMode) { + cacheMode: CacheType.CacheType, + preferredStorageLevel: StorageLevel) + extends Table(tableName, cacheMode, preferredStorageLevel) { // A map from the Hive-partition key to the RDD that contains contents of that partition. private var _keyToPartitions: ConcurrentMap[String, RDD[_]] = @@ -99,11 +107,13 @@ class PartitionedMemoryTable( val loadFunc: String => RDD[_] = (partitionKey: String) => { val partitionRDD = _keyToPartitions.get(partitionKey).get - partitionRDD.persist(RDDUtils.getStorageLevelOfCachedRDDs(getAllPartitions)) + if (partitionRDD.getStorageLevel == StorageLevel.NONE) { + partitionRDD.persist(preferredStorageLevel) + } partitionRDD } val evictionFunc: (String, RDD[_]) => Unit = - (partitionKey: String, partition: RDD[_]) => RDDUtils.unpersistRDD(partition) + (partitionKey: String, partitionRDD: RDD[_]) => RDDUtils.unpersistRDD(partitionRDD) _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) _cachePolicyName = cachePolicyStr } @@ -116,6 +126,9 @@ class PartitionedMemoryTable( def getAllPartitionKeys = _keyToPartitions.keys.toSeq - def getStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfCachedRDDs(getAllPartitions) -} + override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel + override def getCurrentStorageLevel: StorageLevel = { + return RDDUtils.getStorageLevelOfCachedRDDs(getAllPartitions) + } +} diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index a4cef558..316749ce 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -178,11 +178,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey(qb) } - val storageLevel = table.getStorageLevel + val preferredStorageLevel = table.getPreferredStorageLevel OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, cachedTableName, - storageLevel, + preferredStorageLevel, _resSchema.size, /* numColumns */ hivePartitionKey, cacheMode, @@ -202,13 +202,13 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with Seq { if (qb.isCTAS && qb.getTableDesc != null && CacheType.shouldCache(qb.getCacheModeForCreateTable())) { - val storageLevel = MemoryMetadataManager.getStorageLevelFromString( + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( qb.getTableDesc().getTblProps.get("shark.cache.storageLevel")) qb.getTableDesc().getTblProps().put(CachedTableRecovery.QUERY_STRING, ctx.getCmd()) OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOps.head, qb.getTableDesc.getTableName, - storageLevel, + preferredStorageLevel, _resSchema.size, /* numColumns */ new String, /* hivePartitionKey */ qb.getCacheModeForCreateTable, From 7883ea486faac3dadd17eca6fc0bd6e72de6bb80 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 20:53:37 -0700 Subject: [PATCH 101/331] Put LRUCachePolicy in its own file. --- .../scala/shark/memstore2/CachePolicy.scala | 72 --------------- .../shark/memstore2/LRUCachePolicy.scala | 92 +++++++++++++++++++ 2 files changed, 92 insertions(+), 72 deletions(-) create mode 100644 src/main/scala/shark/memstore2/LRUCachePolicy.scala diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index 1a74ed7c..b78a500c 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -17,9 +17,6 @@ package shark.memstore2 -import com.google.common.cache._ -import scala.collection.JavaConversions._ - trait CachePolicy[K, V] { @@ -54,72 +51,3 @@ trait CachePolicy[K, V] { def getEvictionCount: Option[Long] = None } - -class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { - - var isInitialized = false - var hasRecordedStats = false - var cache: LoadingCache[K, V] = _ - var cacheStats: Option[CacheStats] = None - - override def initialize( - maxSize: Long, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit, - shouldRecordStats: Boolean - ): Unit = { - super.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) - - var builder = CacheBuilder.newBuilder().maximumSize(maxSize) - if (shouldRecordStats) { - builder.recordStats() - hasRecordedStats = true - } - - val removalListener = - new RemovalListener[K, V] { - def onRemoval(removal: RemovalNotification[K, V]): Unit = { - evictionFunc(removal.getKey, removal.getValue) - } - } - val cacheLoader = - new CacheLoader[K, V] { - def load(key: K): V = loadFunc(key) - } - - cache = builder - .removalListener(removalListener) - .build(cacheLoader) - isInitialized = true - } - - override def notifyPut(key: K, value: V): Unit = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - cache.put(key, value) - } - - override def notifyRemove(key: K, value: V): Unit = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - cache.invalidate(key, value) - } - - override def notifyGet(key: K): Unit = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - cache.get(key) - } - - override def getKeysOfCachedEntries: Seq[K] = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - return cache.asMap.keySet.toSeq - } - - override def getHitRate(): Option[Double] = { - val hitRate = if (hasRecordedStats) Some(cache.stats.hitRate) else None - return hitRate - } - - override def getEvictionCount(): Option[Long] = { - val evictionCount = if (hasRecordedStats) Some(cache.stats.evictionCount) else None - return evictionCount - } -} diff --git a/src/main/scala/shark/memstore2/LRUCachePolicy.scala b/src/main/scala/shark/memstore2/LRUCachePolicy.scala new file mode 100644 index 00000000..8c8675b3 --- /dev/null +++ b/src/main/scala/shark/memstore2/LRUCachePolicy.scala @@ -0,0 +1,92 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import com.google.common.cache._ + +import scala.collection.JavaConversions._ + + +class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { + + var isInitialized = false + var hasRecordedStats = false + var cache: LoadingCache[K, V] = _ + var cacheStats: Option[CacheStats] = None + + override def initialize( + maxSize: Long, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit, + shouldRecordStats: Boolean + ): Unit = { + super.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) + + var builder = CacheBuilder.newBuilder().maximumSize(maxSize) + if (shouldRecordStats) { + builder.recordStats() + hasRecordedStats = true + } + + val removalListener = + new RemovalListener[K, V] { + def onRemoval(removal: RemovalNotification[K, V]): Unit = { + evictionFunc(removal.getKey, removal.getValue) + } + } + val cacheLoader = + new CacheLoader[K, V] { + def load(key: K): V = loadFunc(key) + } + + cache = builder + .removalListener(removalListener) + .build(cacheLoader) + isInitialized = true + } + + override def notifyPut(key: K, value: V): Unit = { + assert(isInitialized, "Must initialize() LRUCachePolicy.") + cache.put(key, value) + } + + override def notifyRemove(key: K, value: V): Unit = { + assert(isInitialized, "Must initialize() LRUCachePolicy.") + cache.invalidate(key, value) + } + + override def notifyGet(key: K): Unit = { + assert(isInitialized, "Must initialize() LRUCachePolicy.") + cache.get(key) + } + + override def getKeysOfCachedEntries: Seq[K] = { + assert(isInitialized, "Must initialize() LRUCachePolicy.") + return cache.asMap.keySet.toSeq + } + + override def getHitRate(): Option[Double] = { + val hitRate = if (hasRecordedStats) Some(cache.stats.hitRate) else None + return hitRate + } + + override def getEvictionCount(): Option[Long] = { + val evictionCount = if (hasRecordedStats) Some(cache.stats.evictionCount) else None + return evictionCount + } +} From 4db01cee6d673ae37b151f3dc4ee1950b8abf8a6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 6 Oct 2013 23:38:08 -0700 Subject: [PATCH 102/331] Properly account for INSERT INTO/OVERWRITE. Use RDDUtils to find StorageLevel, due to non-persistence of UnionRDDs. --- .../execution/MemoryStoreSinkOperator.scala | 12 +- src/main/scala/shark/execution/RDDUtils.scala | 8 +- .../scala/shark/execution/SparkDDLTask.scala | 3 +- .../memstore2/MemoryMetadataManager.scala | 2 +- .../scala/shark/memstore2/MemoryTable.scala | 72 +++++++---- src/test/scala/shark/CachePolicySuite.scala | 116 ++++++++++++------ 6 files changed, 140 insertions(+), 73 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 6bbdc49e..ef2d8a5a 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -98,6 +98,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { } val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned(tableName) + var hasPreviousRDD = false if (tachyonWriter != null) { // Put the table in Tachyon. @@ -139,7 +140,10 @@ class MemoryStoreSinkOperator extends TerminalOperator { rdd = oldRdd match { case Some(definedRdd) => rdd.union(oldRdd.get.asInstanceOf[RDD[TablePartition]]) // The oldRdd can be missing if this is an INSERT into a new Hive-partition. - case None => rdd + case None => { + hasPreviousRDD = true + rdd + } } } // Run a job on the original RDD to force it to go into cache. @@ -148,8 +152,12 @@ class MemoryStoreSinkOperator extends TerminalOperator { if (isHivePartitioned) { SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).foreach{ table => - table.putPartition(hivePartitionKey, rdd) rdd.setName(tableName + "(" + hivePartitionKey + ")") + if (useUnionRDD && !hasPreviousRDD) { + table.updatePartition(hivePartitionKey, rdd) + } else { + table.putPartition(hivePartitionKey, rdd) + } } } else { val table = SharkEnv.memoryMetadataManager.getMemoryTable(tableName).getOrElse( diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 4d93227a..c94b67a1 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -34,18 +34,18 @@ import shark.SharkEnv */ object RDDUtils { - def getStorageLevelOfCachedRDD(rdd: RDD[_]): StorageLevel = { + def getStorageLevelOfRDD(rdd: RDD[_]): StorageLevel = { rdd match { - case u: UnionRDD[_] => getStorageLevelOfCachedRDDs(u.rdds) + case u: UnionRDD[_] => getStorageLevelOfRDDs(u.rdds) case _ => rdd.getStorageLevel } } - def getStorageLevelOfCachedRDDs(rdds: Seq[RDD[_]]): StorageLevel = { + def getStorageLevelOfRDDs(rdds: Seq[RDD[_]]): StorageLevel = { rdds.foldLeft(StorageLevel.NONE) { (s, r) => { if (s == StorageLevel.NONE) { - getStorageLevelOfCachedRDD(r) + getStorageLevelOfRDD(r) } else { s } diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 0f8213b5..660a75c0 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -77,7 +77,8 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab SharkConfVars.CACHE_POLICY.defaultVal) val maxCacheSize = tblProps.getOrElse("shark.cache.partition.cachePolicy.maxSize", SharkConfVars.MAX_CACHE_SIZE.defaultVal).toLong - val shouldRecordStats = tblProps.getOrElse("shark.cache.partition.shouldRecordStats", + val shouldRecordStats = tblProps.getOrElse( + "shark.cache.partition.cachePolicy.shouldRecordStats", SharkConfVars.SHOULD_RECORD_CACHE_STATS.defaultBoolVal.toString).toBoolean SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( tableName, diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index c1aca720..53e82b11 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -152,7 +152,7 @@ object MemoryMetadataManager { if (table.isInstanceOf[PartitionedMemoryTable]) { val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] // unpersist() all RDDs for all Hive-partitions. - val unpersistedRDDs = partitionedTable.getAllPartitions.map( + val unpersistedRDDs = partitionedTable.keyToPartitions.values.map( rdd => RDDUtils.unpersistRDD(rdd)).asInstanceOf[Seq[RDD[Any]]] if (unpersistedRDDs.size > 0) { val unionedRDD = new UnionRDD(unpersistedRDDs.head.context, unpersistedRDDs) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index cd3279ea..f270a7ba 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -53,11 +53,15 @@ class MemoryTable( extends Table(tableName, cacheMode, preferredStorageLevel) { // RDD that contains the contents of this table. - var tableRDD: RDD[_] = _ + private var _tableRDD: RDD[_] = _ + + def tableRDD: RDD[_] = _tableRDD + + def tableRDD_= (rdd: RDD[_]) = _tableRDD = rdd override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel - override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfCachedRDD(tableRDD) + override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfRDD(tableRDD) } private[shark] @@ -67,34 +71,52 @@ class PartitionedMemoryTable( preferredStorageLevel: StorageLevel) extends Table(tableName, cacheMode, preferredStorageLevel) { + private class RDDValue(var rdd: RDD[_]) + // A map from the Hive-partition key to the RDD that contains contents of that partition. - private var _keyToPartitions: ConcurrentMap[String, RDD[_]] = - new ConcurrentJavaHashMap[String, RDD[_]]() + private var _keyToPartitions: ConcurrentMap[String, RDDValue] = + new ConcurrentJavaHashMap[String, RDDValue]() // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. - private var _cachePolicy: CachePolicy[String, RDD[_]] = _ + private var _cachePolicy: CachePolicy[String, RDDValue] = _ private var _cachePolicyName: String = "None" def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) def getPartition(partitionKey: String): Option[RDD[_]] = { - val rddFound = _keyToPartitions.get(partitionKey) - if (rddFound.isDefined) _cachePolicy.notifyGet(partitionKey) - return rddFound + val rddValueFound = _keyToPartitions.get(partitionKey) + if (rddValueFound.isDefined) _cachePolicy.notifyGet(partitionKey) + return rddValueFound.map(_.rdd) } - def putPartition(partitionKey: String, rdd: RDD[_]): Option[RDD[_]] = { - _cachePolicy.notifyPut(partitionKey, rdd) - _keyToPartitions.put(partitionKey, rdd) + def updatePartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { + val rddValueFound = _keyToPartitions.get(partitionKey) + var oldRDD: Option[RDD[_]] = None + if (rddValueFound.isDefined) { + val reusedRDDValue = rddValueFound.get + oldRDD = Some(reusedRDDValue.rdd) + reusedRDDValue.rdd = newRDD + reusedRDDValue + } + return oldRDD + } + + def putPartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { + val rddValueFound = _keyToPartitions.get(partitionKey) + var oldRDD: Option[RDD[_]] = rddValueFound.map(_.rdd) + val newRDDValue = new RDDValue(newRDD) + _keyToPartitions.put(partitionKey, newRDDValue) + _cachePolicy.notifyPut(partitionKey, newRDDValue) + return oldRDD } def removePartition(partitionKey: String): Option[RDD[_]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey, rddRemoved.get) - return rddRemoved + return rddRemoved.map(_.rdd) } def setPartitionCachePolicy( @@ -103,32 +125,32 @@ class PartitionedMemoryTable( shouldRecordStats: Boolean ) { _cachePolicy = - Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDD[_]]] - val loadFunc: String => RDD[_] = + Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDDValue]] + val loadFunc: String => RDDValue = (partitionKey: String) => { - val partitionRDD = _keyToPartitions.get(partitionKey).get - if (partitionRDD.getStorageLevel == StorageLevel.NONE) { - partitionRDD.persist(preferredStorageLevel) + val rddValue = _keyToPartitions.get(partitionKey).get + if (RDDUtils.getStorageLevelOfRDD(rddValue.rdd) == StorageLevel.NONE) { + rddValue.rdd.persist(preferredStorageLevel) } - partitionRDD + rddValue } - val evictionFunc: (String, RDD[_]) => Unit = - (partitionKey: String, partitionRDD: RDD[_]) => RDDUtils.unpersistRDD(partitionRDD) + val evictionFunc: (String, RDDValue) => Unit = + (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) _cachePolicyName = cachePolicyStr } def cachePolicyName: String = _cachePolicyName - def cachePolicy: CachePolicy[String, RDD[_]] = _cachePolicy - - def getAllPartitions = _keyToPartitions.values.toSeq + def cachePolicy: CachePolicy[String, _] = _cachePolicy - def getAllPartitionKeys = _keyToPartitions.keys.toSeq + def keyToPartitions: collection.immutable.Map[String, RDD[_]] = { + return _keyToPartitions.mapValues(_.rdd).toMap + } override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel override def getCurrentStorageLevel: StorageLevel = { - return RDDUtils.getStorageLevelOfCachedRDDs(getAllPartitions) + return RDDUtils.getStorageLevelOfRDDs(_keyToPartitions.values.map(_.rdd).toSeq) } } diff --git a/src/test/scala/shark/CachePolicySuite.scala b/src/test/scala/shark/CachePolicySuite.scala index d537f400..ca6b3cba 100644 --- a/src/test/scala/shark/CachePolicySuite.scala +++ b/src/test/scala/shark/CachePolicySuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.PartitionedMemoryTable +import shark.execution.RDDUtils class CachePolicySuite extends FunSuite with BeforeAndAfterAll { @@ -76,8 +77,8 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { maxSize, cachePolicyClassName, shouldRecordStats)) - var partitionNum = 0 - while (partitionNum < numPartitionsToCreate) { + var partitionNum = 1 + while (partitionNum <= numPartitionsToCreate) { sc.runSql("""insert into table %s partition(keypart = %d) select * from test_cached""".format(tableName, partitionNum)) partitionNum += 1 @@ -104,32 +105,33 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { 3 /* maxSize */, 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") - val keypart1RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("""insert into table evict_partitions_maxSize partition(keypart = 4) select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for get()s.") { + test("LRU: RDD eviction accounts for partition scans - i.e., a cache.get()") { val tableName = "evict_partitions_with_get" val partitionedTable = createCachedPartitionedTable( tableName, 3 /* maxSize */, 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") - val keypart1RDD = partitionedTable.getPartition("keypart=1") - val keypart2RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("select count(1) from evict_partitions_with_get where keypart = 1") sc.runSql("""insert into table evict_partitions_with_get partition(keypart = 4) select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for put()s.") { + test("LRU: RDD eviction accounts for INSERT INTO - i.e., a cache.get().") { val tableName = "evict_partitions_with_put" val partitionedTable = createCachedPartitionedTable( tableName, @@ -137,16 +139,44 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val keypart1RDD = partitionedTable.getPartition("keypart=1") - val keypart2RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 1) select * from test""") sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for INSERT OVERWRITE - i.e. a cache.put()") { + val tableName = "evict_partitions_with_put" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* maxSize */, + 3 /* numPartitionsToCreate */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert overwrite table evict_partitions_with_put partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + select * from test""") + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) + val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE) } test("LRU: get() reloads an RDD previously unpersist()'d.") { @@ -157,37 +187,24 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { 3 /* numPartitionsToCreate */, "shark.memstore2.LRUCachePolicy") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val keypart1RDD = partitionedTable.getPartition("keypart=1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) select * from test""") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and // cause eviction of the RDD for partition (keypart = 2). sc.runSql("select count(1) from reload_evicted_partition where keypart = 1") assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - val keypart2RDD = partitionedTable.getPartition("keypart=1") - assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) - } - - test("LRU: cache stats are not recorded by default") { - val tableName = "dont_record_partition_cache_stats" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* maxSize */, - 1 /* numPartitionsToCreate */, - "shark.memstore2.LRUCachePolicy", - true /* shouldRecordStats */) - val lruCachePolicy = partitionedTable.cachePolicy - val hitRate = lruCachePolicy.getHitRate - assert(hitRate.isEmpty) - val evictionCount = lruCachePolicy.getEvictionCount - assert(evictionCount.isEmpty) + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE, + "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) } test("LRU: record cache stats if user specifies it") { - val tableName = "record_partition_cache_stats" + val tableName = "should_record_partition_cache_stats" val partitionedTable = createCachedPartitionedTable( tableName, 3 /* maxSize */, @@ -202,4 +219,23 @@ class CachePolicySuite extends FunSuite with BeforeAndAfterAll { assert(evictionCount.isDefined) assert(evictionCount.get == 0) } + + test("LRU: cache stats are not recorded by default") { + sc.runSql("drop table if exists dont_record_partition_cache_stats") + sc.runSql(""" + create table dont_record_partition_cache_stats(key int, value string) + partitioned by (keypart int) + tblproperties( + 'shark.cache' = 'true', + 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy') + """) + val tableName = "dont_record_partition_cache_stats" + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val lruCachePolicy = partitionedTable.cachePolicy + val hitRate = lruCachePolicy.getHitRate + assert(hitRate.isEmpty) + val evictionCount = lruCachePolicy.getEvictionCount + assert(evictionCount.isEmpty) + } } From bc94e37354648989b95ebe19adcf555b976be4fc Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 01:08:08 -0700 Subject: [PATCH 103/331] Move CachePolicySuite tests back to SQLSuite. Also, gave clearer names to partition-caching table properties. --- src/main/scala/shark/SharkConfVars.scala | 4 +- .../scala/shark/execution/SparkDDLTask.scala | 4 +- src/test/scala/shark/CachePolicySuite.scala | 241 ----------------- src/test/scala/shark/SQLSuite.scala | 252 +++++++++++++++--- 4 files changed, 217 insertions(+), 284 deletions(-) delete mode 100644 src/test/scala/shark/CachePolicySuite.scala diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index c64f1acc..0b351158 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -48,10 +48,10 @@ object SharkConfVars { "shark.cache.partition.cachePolicy.class", "shark.memstore2.LRUCachePolicy") // Max size of the cache specified by the "shark.cache.partition.cachePolicy" property. - val MAX_CACHE_SIZE = new ConfVar("shark.cache.partition.cachePolicy.maxSize", "10") + val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.partition.cachePolicy.maxSize", "10") // Whether the cache specified by "shark.cache.partition.cachePolicy" should record statistics. - val SHOULD_RECORD_CACHE_STATS = new ConfVar( + val SHOULD_RECORD_PARTITION_CACHE_STATS = new ConfVar( "shark.cache.partition.cachePolicy.shouldRecordStats", false) // If true, then cache any table whose name ends in "_cached". diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SparkDDLTask.scala index 660a75c0..5b05193d 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SparkDDLTask.scala @@ -76,10 +76,10 @@ private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializab val cachePolicyStr = tblProps.getOrElse("shark.cache.partition.cachePolicy.class", SharkConfVars.CACHE_POLICY.defaultVal) val maxCacheSize = tblProps.getOrElse("shark.cache.partition.cachePolicy.maxSize", - SharkConfVars.MAX_CACHE_SIZE.defaultVal).toLong + SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toLong val shouldRecordStats = tblProps.getOrElse( "shark.cache.partition.cachePolicy.shouldRecordStats", - SharkConfVars.SHOULD_RECORD_CACHE_STATS.defaultBoolVal.toString).toBoolean + SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.defaultBoolVal.toString).toBoolean SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( tableName, cacheMode, diff --git a/src/test/scala/shark/CachePolicySuite.scala b/src/test/scala/shark/CachePolicySuite.scala deleted file mode 100644 index ca6b3cba..00000000 --- a/src/test/scala/shark/CachePolicySuite.scala +++ /dev/null @@ -1,241 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.spark.storage.StorageLevel - -import shark.api.QueryExecutionException -import shark.memstore2.PartitionedMemoryTable -import shark.execution.RDDUtils - - -class CachePolicySuite extends FunSuite with BeforeAndAfterAll { - - val WAREHOUSE_PATH = TestUtils.getWarehousePath() - val METASTORE_PATH = TestUtils.getMetastorePath() - val MASTER = "local" - - var sc: SharkContext = _ - - override def beforeAll() { - sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) - sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + - METASTORE_PATH + ";create=true") - sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) - - // test - sc.runSql("drop table if exists test") - sc.runSql("CREATE TABLE test (key INT, val STRING)") - sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test") - sc.runSql("drop table if exists test_cached") - sc.runSql("CREATE TABLE test_cached AS SELECT * FROM test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - private def createCachedPartitionedTable( - tableName: String, - maxSize: Int, - numPartitionsToCreate: Int, - cachePolicyClassName: String, - shouldRecordStats: Boolean = false - ): PartitionedMemoryTable = { - sc.runSql("drop table if exists %s".format(tableName)) - sc.runSql(""" - create table %s(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache' = 'true', - 'shark.cache.partition.cachePolicy.maxSize' = '%d', - 'shark.cache.partition.cachePolicy.class' = '%s', - 'shark.cache.storageLevel' = 'MEMORY_AND_DISK', - 'shark.cache.partition.cachePolicy.shouldRecordStats' = '%b') - """.format( - tableName, - maxSize, - cachePolicyClassName, - shouldRecordStats)) - var partitionNum = 1 - while (partitionNum <= numPartitionsToCreate) { - sc.runSql("""insert into table %s partition(keypart = %d) - select * from test_cached""".format(tableName, partitionNum)) - partitionNum += 1 - } - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - return partitionedTable - } - - test("shark.memstore2.LRUCachePolicy is the default policy") { - val tableName = "lru_default_policy_cached" - sc.runSql("""create table lru_default_policy_cached(key int, value string) - partitioned by (keypart int)""") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val cachePolicy = partitionedTable.cachePolicy - assert(cachePolicy.isInstanceOf[shark.memstore2.LRUCachePolicy[_, _]]) - } - - test("LRU: RDDs are evicted when the max size is reached.") { - val tableName = "evict_partitions_maxSize" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* maxSize */, - 3 /* numPartitionsToCreate */, - "shark.memstore2.LRUCachePolicy") - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_maxSize partition(keypart = 4) - select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for partition scans - i.e., a cache.get()") { - val tableName = "evict_partitions_with_get" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* maxSize */, - 3 /* numPartitionsToCreate */, - "shark.memstore2.LRUCachePolicy") - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("select count(1) from evict_partitions_with_get where keypart = 1") - sc.runSql("""insert into table evict_partitions_with_get partition(keypart = 4) - select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for INSERT INTO - i.e., a cache.get().") { - val tableName = "evict_partitions_with_put" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* maxSize */, - 3 /* numPartitionsToCreate */, - "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) - select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - - val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) - assert(keypart2StorageLevel == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for INSERT OVERWRITE - i.e. a cache.put()") { - val tableName = "evict_partitions_with_put" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* maxSize */, - 3 /* numPartitionsToCreate */, - "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert overwrite table evict_partitions_with_put partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) - select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) - val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - - val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) - assert(keypart2StorageLevel == StorageLevel.NONE) - } - - test("LRU: get() reloads an RDD previously unpersist()'d.") { - val tableName = "reload_evicted_partition" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* maxSize */, - 3 /* numPartitionsToCreate */, - "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) - select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) - - // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and - // cause eviction of the RDD for partition (keypart = 2). - sc.runSql("select count(1) from reload_evicted_partition where keypart = 1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) - assert(keypart2StorageLevel == StorageLevel.NONE, - "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) - } - - test("LRU: record cache stats if user specifies it") { - val tableName = "should_record_partition_cache_stats" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* maxSize */, - 1 /* numPartitionsToCreate */, - "shark.memstore2.LRUCachePolicy", - true /* shouldRecordStats */) - val lruCachePolicy = partitionedTable.cachePolicy - val hitRate = lruCachePolicy.getHitRate - assert(hitRate.isDefined) - assert(hitRate.get == 1.0) - val evictionCount = lruCachePolicy.getEvictionCount - assert(evictionCount.isDefined) - assert(evictionCount.get == 0) - } - - test("LRU: cache stats are not recorded by default") { - sc.runSql("drop table if exists dont_record_partition_cache_stats") - sc.runSql(""" - create table dont_record_partition_cache_stats(key int, value string) - partitioned by (keypart int) - tblproperties( - 'shark.cache' = 'true', - 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy') - """) - val tableName = "dont_record_partition_cache_stats" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val lruCachePolicy = partitionedTable.cachePolicy - val hitRate = lruCachePolicy.getHitRate - assert(hitRate.isEmpty) - val evictionCount = lruCachePolicy.getEvictionCount - assert(evictionCount.isEmpty) - } -} diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 35610340..c67f2788 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -23,7 +23,8 @@ import org.scalatest.FunSuite import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException - +import shark.memstore2.PartitionedMemoryTable +import shark.execution.RDDUtils class SQLSuite extends FunSuite with BeforeAndAfterAll { @@ -103,6 +104,38 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { expectSql(sql, Array(expectedResult)) } + private def createCachedPartitionedTable( + tableName: String, + numPartitionsToCreate: Int, + maxCacheSize: Int = 10, + cachePolicyClassName: String = "shark.memstore2.LRUCachePolicy", + shouldRecordStats: Boolean = false + ): PartitionedMemoryTable = { + sc.runSql("drop table if exists %s".format(tableName)) + sc.runSql(""" + create table %s(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache' = 'true', + 'shark.cache.partition.cachePolicy.maxSize' = '%d', + 'shark.cache.partition.cachePolicy.class' = '%s', + 'shark.cache.storageLevel' = 'MEMORY_AND_DISK', + 'shark.cache.partition.cachePolicy.shouldRecordStats' = '%b') + """.format( + tableName, + maxCacheSize, + cachePolicyClassName, + shouldRecordStats)) + var partitionNum = 1 + while (partitionNum <= numPartitionsToCreate) { + sc.runSql("""insert into table %s partition(keypart = %d) + select * from test_cached""".format(tableName, partitionNum)) + partitionNum += 1 + } + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + return partitionedTable + } + ////////////////////////////////////////////////////////////////////////////// // basic SQL ////////////////////////////////////////////////////////////////////////////// @@ -397,20 +430,19 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("insert into a partition of a cached table") { - sc.runSql("drop table if exists insert_part_cached") - sc.runSql("""create table insert_part_cached(key int, value string) - partitioned by (keypart int)""") - sc.runSql("insert into table insert_part_cached partition(keypart = 1) select * from test") + val tableName = "insert_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) expectSql("select value from insert_part_cached where key = 407 and keypart = 1", "val_407") } test("insert overwrite a partition of a cached table") { - sc.runSql("drop table if exists insert_over_part_cached") - sc.runSql("""create table insert_over_part_cached(key int, value string) - partitioned by (keypart int)""") - sc.runSql("""insert into table insert_over_part_cached partition(keypart = 1) - select * from test""") + val tableName = "insert_over_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) expectSql("""select value from insert_over_part_cached where key = 407 and keypart = 1""", "val_407") sc.runSql("""insert overwrite table insert_over_part_cached partition(keypart = 1) @@ -426,55 +458,46 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("scan cached, partitioned table that has a single partition") { - sc.runSql("drop table if exists scan_single_part_cached") - sc.runSql("""create table scan_single_part_cached(key int, value string) - partitioned by (keypart int)""") - sc.runSql("insert into table scan_single_part_cached partition(keypart = 1) select * from test") + val tableName = "scan_single_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) expectSql("select * from scan_single_part_cached where key = 407", "407\tval_407\t1") } test("scan cached, partitioned table that has multiple partitions") { - sc.runSql("drop table if exists scan_mult_part_cached") - sc.runSql("""create table scan_mult_part_cached(key int, value string) - partitioned by (keypart int)""") - sc.runSql("insert into table scan_mult_part_cached partition(keypart = 1) select * from test") - sc.runSql("insert into table scan_mult_part_cached partition(keypart = 5) select * from test") - sc.runSql("insert into table scan_mult_part_cached partition(keypart = 9) select * from test") + val tableName = "scan_mult_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */) expectSql("select * from scan_mult_part_cached where key = 407 order by keypart", - Array("407\tval_407\t1", "407\tval_407\t5", "407\tval_407\t9")) + Array("407\tval_407\t1", "407\tval_407\t2", "407\tval_407\t3")) } test("drop/unpersist cached, partitioned table that has multiple partitions") { - sc.runSql("drop table if exists drop_mult_part_cached") - sc.runSql("""create table drop_mult_part_cached(key int, value string) - partitioned by (keypart int)""") - sc.runSql("insert into table drop_mult_part_cached partition(keypart = 1) select * from test") - sc.runSql("insert into table drop_mult_part_cached partition(keypart = 5) select * from test") - sc.runSql("insert into table drop_mult_part_cached partition(keypart = 9) select * from test") val tableName = "drop_mult_part_cached" - assert(SharkEnv.memoryMetadataManager.containsTable("drop_mult_part_cached")) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */) val keypart1RDD = partitionedTable.getPartition("keypart=1") - val keypart5RDD = partitionedTable.getPartition("keypart=5") - val keypart9RDD = partitionedTable.getPartition("keypart=9") + val keypart2RDD = partitionedTable.getPartition("keypart=2") + val keypart3RDD = partitionedTable.getPartition("keypart=3") sc.runSql("drop table drop_mult_part_cached ") assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) // All RDDs should have been unpersisted. assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) - assert(keypart5RDD.get.getStorageLevel == StorageLevel.NONE) - assert(keypart9RDD.get.getStorageLevel == StorageLevel.NONE) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + assert(keypart3RDD.get.getStorageLevel == StorageLevel.NONE) } test("drop cached partition represented by a UnionRDD (i.e., the result of multiple inserts)") { - sc.runSql("drop table if exists drop_union_part_cached") - sc.runSql("""create table drop_union_part_cached(key int, value string) - partitioned by (keypart int)""") + val tableName = "drop_union_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") - val tableName = "drop_union_part_cached" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get val keypart1RDD = partitionedTable.getPartition("keypart=1") sc.runSql("drop table drop_union_part_cached") assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) @@ -486,6 +509,157 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // RDD(partition) eviction policy for cached Hive-partititioned tables ////////////////////////////////////////////////////////////////////////////// + test("shark.memstore2.LRUCachePolicy is the default policy") { + val tableName = "lru_default_policy_cached" + sc.runSql("""create table lru_default_policy_cached(key int, value string) + partitioned by (keypart int)""") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val cachePolicy = partitionedTable.cachePolicy + assert(cachePolicy.isInstanceOf[shark.memstore2.LRUCachePolicy[_, _]]) + } + + test("LRU: RDDs are evicted when the max size is reached.") { + val tableName = "evict_partitions_maxSize" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table evict_partitions_maxSize partition(keypart = 4) + select * from test""") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for partition scans - i.e., a cache.get()") { + val tableName = "evict_partitions_with_get" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("select count(1) from evict_partitions_with_get where keypart = 1") + sc.runSql("""insert into table evict_partitions_with_get partition(keypart = 4) + select * from test""") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for INSERT INTO - i.e., a cache.get().") { + val tableName = "evict_partitions_with_put" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + select * from test""") + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for INSERT OVERWRITE - i.e. a cache.put()") { + val tableName = "evict_partitions_with_put" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert overwrite table evict_partitions_with_put partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + select * from test""") + assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) + val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE) + } + + test("LRU: get() reloads an RDD previously unpersist()'d.") { + val tableName = "reload_evicted_partition" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) + select * from test""") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) + + // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and + // cause eviction of the RDD for partition (keypart = 2). + sc.runSql("select count(1) from reload_evicted_partition where keypart = 1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE, + "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) + } + + test("LRU: record cache stats if user specifies it") { + val tableName = "should_record_partition_cache_stats" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy", + true /* shouldRecordStats */) + val lruCachePolicy = partitionedTable.cachePolicy + val hitRate = lruCachePolicy.getHitRate + assert(hitRate.isDefined) + assert(hitRate.get == 1.0) + val evictionCount = lruCachePolicy.getEvictionCount + assert(evictionCount.isDefined) + assert(evictionCount.get == 0) + } + + test("LRU: cache stats are not recorded by default") { + sc.runSql("drop table if exists dont_record_partition_cache_stats") + sc.runSql(""" + create table dont_record_partition_cache_stats(key int, value string) + partitioned by (keypart int) + tblproperties( + 'shark.cache' = 'true', + 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy') + """) + val tableName = "dont_record_partition_cache_stats" + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val lruCachePolicy = partitionedTable.cachePolicy + val hitRate = lruCachePolicy.getHitRate + assert(hitRate.isEmpty) + val evictionCount = lruCachePolicy.getEvictionCount + assert(evictionCount.isEmpty) + } + ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// From fe6401eed308d1a73dbcfddae4a62c6cfee485b0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 01:11:04 -0700 Subject: [PATCH 104/331] MemoryTable.scala => Table.scala --- src/main/scala/shark/memstore2/{MemoryTable.scala => Table.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/main/scala/shark/memstore2/{MemoryTable.scala => Table.scala} (100%) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/Table.scala similarity index 100% rename from src/main/scala/shark/memstore2/MemoryTable.scala rename to src/main/scala/shark/memstore2/Table.scala From 48e63bb9af746a6c37edb92a2a4af649e3c9b1af Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 01:25:54 -0700 Subject: [PATCH 105/331] Move MemoryTable and PartitionedMemoryTable to their own files. --- .../scala/shark/memstore2/MemoryTable.scala | 43 +++++++ .../memstore2/PartitionedMemoryTable.scala | 120 ++++++++++++++++++ src/main/scala/shark/memstore2/Table.scala | 118 ----------------- 3 files changed, 163 insertions(+), 118 deletions(-) create mode 100644 src/main/scala/shark/memstore2/MemoryTable.scala create mode 100644 src/main/scala/shark/memstore2/PartitionedMemoryTable.scala diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala new file mode 100644 index 00000000..950e3853 --- /dev/null +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import shark.execution.RDDUtils + +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + + +private[shark] +class MemoryTable( + tableName: String, + cacheMode: CacheType.CacheType, + preferredStorageLevel: StorageLevel) + extends Table(tableName, cacheMode, preferredStorageLevel) { + + // RDD that contains the contents of this table. + private var _tableRDD: RDD[_] = _ + + def tableRDD: RDD[_] = _tableRDD + + def tableRDD_= (rdd: RDD[_]) = _tableRDD = rdd + + override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel + + override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfRDD(tableRDD) +} diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala new file mode 100644 index 00000000..83a2dc5d --- /dev/null +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -0,0 +1,120 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ConcurrentMap + +import shark.execution.RDDUtils + +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + + +private[shark] +class PartitionedMemoryTable( + tableName: String, + cacheMode: CacheType.CacheType, + preferredStorageLevel: StorageLevel) + extends Table(tableName, cacheMode, preferredStorageLevel) { + + private class RDDValue(var rdd: RDD[_]) + + // A map from the Hive-partition key to the RDD that contains contents of that partition. + private var _keyToPartitions: ConcurrentMap[String, RDDValue] = + new ConcurrentJavaHashMap[String, RDDValue]() + + // The eviction policy for this table's cached Hive-partitions. An example of how this + // can be set from the CLI: + // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. + private var _cachePolicy: CachePolicy[String, RDDValue] = _ + + private var _cachePolicyName: String = "None" + + def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) + + def getPartition(partitionKey: String): Option[RDD[_]] = { + val rddValueFound = _keyToPartitions.get(partitionKey) + if (rddValueFound.isDefined) _cachePolicy.notifyGet(partitionKey) + return rddValueFound.map(_.rdd) + } + + def updatePartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { + val rddValueFound = _keyToPartitions.get(partitionKey) + var oldRDD: Option[RDD[_]] = None + if (rddValueFound.isDefined) { + val reusedRDDValue = rddValueFound.get + oldRDD = Some(reusedRDDValue.rdd) + reusedRDDValue.rdd = newRDD + reusedRDDValue + } + return oldRDD + } + + def putPartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { + val rddValueFound = _keyToPartitions.get(partitionKey) + var oldRDD: Option[RDD[_]] = rddValueFound.map(_.rdd) + val newRDDValue = new RDDValue(newRDD) + _keyToPartitions.put(partitionKey, newRDDValue) + _cachePolicy.notifyPut(partitionKey, newRDDValue) + return oldRDD + } + + def removePartition(partitionKey: String): Option[RDD[_]] = { + val rddRemoved = _keyToPartitions.remove(partitionKey) + if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey, rddRemoved.get) + return rddRemoved.map(_.rdd) + } + + def setPartitionCachePolicy( + cachePolicyStr: String, + maxSize: Long, + shouldRecordStats: Boolean + ) { + _cachePolicy = + Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDDValue]] + val loadFunc: String => RDDValue = + (partitionKey: String) => { + val rddValue = _keyToPartitions.get(partitionKey).get + if (RDDUtils.getStorageLevelOfRDD(rddValue.rdd) == StorageLevel.NONE) { + rddValue.rdd.persist(preferredStorageLevel) + } + rddValue + } + val evictionFunc: (String, RDDValue) => Unit = + (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) + _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) + _cachePolicyName = cachePolicyStr + } + + def cachePolicyName: String = _cachePolicyName + + def cachePolicy: CachePolicy[String, _] = _cachePolicy + + def keyToPartitions: collection.immutable.Map[String, RDD[_]] = { + return _keyToPartitions.mapValues(_.rdd).toMap + } + + override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel + + override def getCurrentStorageLevel: StorageLevel = { + return RDDUtils.getStorageLevelOfRDDs(_keyToPartitions.values.map(_.rdd).toSeq) + } +} diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index f270a7ba..a1317394 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -17,16 +17,8 @@ package shark.memstore2 -import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ConcurrentMap - -import org.apache.spark.rdd.{EmptyRDD, RDD} import org.apache.spark.storage.StorageLevel -import shark.execution.RDDUtils - /** * A container for table metadata specific to Shark and Spark. Currently, this is a lightweight @@ -44,113 +36,3 @@ private[shark] abstract class Table( def getCurrentStorageLevel: StorageLevel } - -private[shark] -class MemoryTable( - tableName: String, - cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel) - extends Table(tableName, cacheMode, preferredStorageLevel) { - - // RDD that contains the contents of this table. - private var _tableRDD: RDD[_] = _ - - def tableRDD: RDD[_] = _tableRDD - - def tableRDD_= (rdd: RDD[_]) = _tableRDD = rdd - - override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel - - override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfRDD(tableRDD) -} - -private[shark] -class PartitionedMemoryTable( - tableName: String, - cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel) - extends Table(tableName, cacheMode, preferredStorageLevel) { - - private class RDDValue(var rdd: RDD[_]) - - // A map from the Hive-partition key to the RDD that contains contents of that partition. - private var _keyToPartitions: ConcurrentMap[String, RDDValue] = - new ConcurrentJavaHashMap[String, RDDValue]() - - // The eviction policy for this table's cached Hive-partitions. An example of how this - // can be set from the CLI: - // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. - private var _cachePolicy: CachePolicy[String, RDDValue] = _ - - private var _cachePolicyName: String = "None" - - def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) - - def getPartition(partitionKey: String): Option[RDD[_]] = { - val rddValueFound = _keyToPartitions.get(partitionKey) - if (rddValueFound.isDefined) _cachePolicy.notifyGet(partitionKey) - return rddValueFound.map(_.rdd) - } - - def updatePartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { - val rddValueFound = _keyToPartitions.get(partitionKey) - var oldRDD: Option[RDD[_]] = None - if (rddValueFound.isDefined) { - val reusedRDDValue = rddValueFound.get - oldRDD = Some(reusedRDDValue.rdd) - reusedRDDValue.rdd = newRDD - reusedRDDValue - } - return oldRDD - } - - def putPartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { - val rddValueFound = _keyToPartitions.get(partitionKey) - var oldRDD: Option[RDD[_]] = rddValueFound.map(_.rdd) - val newRDDValue = new RDDValue(newRDD) - _keyToPartitions.put(partitionKey, newRDDValue) - _cachePolicy.notifyPut(partitionKey, newRDDValue) - return oldRDD - } - - def removePartition(partitionKey: String): Option[RDD[_]] = { - val rddRemoved = _keyToPartitions.remove(partitionKey) - if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey, rddRemoved.get) - return rddRemoved.map(_.rdd) - } - - def setPartitionCachePolicy( - cachePolicyStr: String, - maxSize: Long, - shouldRecordStats: Boolean - ) { - _cachePolicy = - Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDDValue]] - val loadFunc: String => RDDValue = - (partitionKey: String) => { - val rddValue = _keyToPartitions.get(partitionKey).get - if (RDDUtils.getStorageLevelOfRDD(rddValue.rdd) == StorageLevel.NONE) { - rddValue.rdd.persist(preferredStorageLevel) - } - rddValue - } - val evictionFunc: (String, RDDValue) => Unit = - (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) - _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) - _cachePolicyName = cachePolicyStr - } - - def cachePolicyName: String = _cachePolicyName - - def cachePolicy: CachePolicy[String, _] = _cachePolicy - - def keyToPartitions: collection.immutable.Map[String, RDD[_]] = { - return _keyToPartitions.mapValues(_.rdd).toMap - } - - override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel - - override def getCurrentStorageLevel: StorageLevel = { - return RDDUtils.getStorageLevelOfRDDs(_keyToPartitions.values.map(_.rdd).toSeq) - } -} From 1746e81e956e0617b97ea95f0910ba362f4bfa6e Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 01:42:36 -0700 Subject: [PATCH 106/331] Unit test for partition removals for LRUCachePolicy. --- src/test/scala/shark/SQLSuite.scala | 35 +++++++++++++++++++++-------- 1 file changed, 26 insertions(+), 9 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index c67f2788..ca8e3449 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -533,7 +533,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for partition scans - i.e., a cache.get()") { + test("LRU: RDD eviction accounts for partition scans - a cache.get()") { val tableName = "evict_partitions_with_get" val partitionedTable = createCachedPartitionedTable( tableName, @@ -552,8 +552,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for INSERT INTO - i.e., a cache.get().") { - val tableName = "evict_partitions_with_put" + test("LRU: RDD eviction accounts for INSERT INTO - a cache.get().") { + val tableName = "evict_partitions_insert_into" val partitionedTable = createCachedPartitionedTable( tableName, 3 /* numPartitionsToCreate */, @@ -564,9 +564,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 1) + sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 1) select * from test""") - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 4) select * from test""") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") @@ -576,8 +576,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(keypart2StorageLevel == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for INSERT OVERWRITE - i.e. a cache.put()") { - val tableName = "evict_partitions_with_put" + test("LRU: RDD eviction accounts for INSERT OVERWRITE - a cache.put()") { + val tableName = "evict_partitions_insert_overwrite" val partitionedTable = createCachedPartitionedTable( tableName, 3 /* numPartitionsToCreate */, @@ -588,9 +588,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert overwrite table evict_partitions_with_put partition(keypart = 1) + sc.runSql("""insert overwrite table evict_partitions_insert_overwrite partition(keypart = 1) select * from test""") - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + sc.runSql("""insert into table evict_partitions_insert_overwrite partition(keypart = 4) select * from test""") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") @@ -600,6 +600,23 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(keypart2StorageLevel == StorageLevel.NONE) } + test("LRU: RDD eviction accounts for ALTER TABLE DROP PARTITION - a cache.remove()") { + val tableName = "evict_partitions_removals" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + sc.runSql("alter table evict_partitions_removals drop partition(keypart = 1)") + sc.runSql("""insert into table evict_partitions_removals partition(keypart = 4) + select * from test""") + sc.runSql("""insert into table evict_partitions_removals partition(keypart = 5) + select * from test""") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) + } + test("LRU: get() reloads an RDD previously unpersist()'d.") { val tableName = "reload_evicted_partition" val partitionedTable = createCachedPartitionedTable( From 5cea58c6e35ab41746c200570d845d2445f83f57 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 01:42:36 -0700 Subject: [PATCH 107/331] Unit test for partition removals for LRUCachePolicy. --- src/main/scala/shark/SharkConfVars.scala | 3 +- src/main/scala/shark/memstore2/Table.scala | 3 +- src/test/scala/shark/SQLSuite.scala | 35 ++++++++++++++++------ 3 files changed, 30 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 0b351158..b3eb400b 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -76,7 +76,8 @@ object SharkConfVars { conf.set(EXPLAIN_MODE.varname, EXPLAIN_MODE.defaultVal) } if (conf.get(COLUMN_BUILDER_PARTITION_SIZE.varname) == null) { - conf.setInt(COLUMN_BUILDER_PARTITION_SIZE.varname, COLUMN_BUILDER_PARTITION_SIZE.defaultIntVal) + conf.setInt( + COLUMN_BUILDER_PARTITION_SIZE.varname,COLUMN_BUILDER_PARTITION_SIZE.defaultIntVal) } if (conf.get(COLUMNAR_COMPRESSION.varname) == null) { conf.setBoolean(COLUMNAR_COMPRESSION.varname, COLUMNAR_COMPRESSION.defaultBoolVal) diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index a1317394..9a38a733 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -23,9 +23,10 @@ import org.apache.spark.storage.StorageLevel /** * A container for table metadata specific to Shark and Spark. Currently, this is a lightweight * wrapper around either an RDD or multiple RDDs if the Shark table is Hive-partitioned. + * * Note that a Hive-partition of a table is different from an RDD partition. Each Hive-partition * is stored as a subdirectory of the table subdirectory in the warehouse directory - * (e.g. /user/hive/warehouse). So, every Hive-Partition is loaded into Shark as an RDD. + * (e.g. '/user/hive/warehouse'). So, every Hive-Partition is loaded into Shark as an RDD. */ private[shark] abstract class Table( var tableName: String, diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index c67f2788..ca8e3449 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -533,7 +533,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for partition scans - i.e., a cache.get()") { + test("LRU: RDD eviction accounts for partition scans - a cache.get()") { val tableName = "evict_partitions_with_get" val partitionedTable = createCachedPartitionedTable( tableName, @@ -552,8 +552,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for INSERT INTO - i.e., a cache.get().") { - val tableName = "evict_partitions_with_put" + test("LRU: RDD eviction accounts for INSERT INTO - a cache.get().") { + val tableName = "evict_partitions_insert_into" val partitionedTable = createCachedPartitionedTable( tableName, 3 /* numPartitionsToCreate */, @@ -564,9 +564,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 1) + sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 1) select * from test""") - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 4) select * from test""") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") @@ -576,8 +576,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(keypart2StorageLevel == StorageLevel.NONE) } - test("LRU: RDD eviction accounts for INSERT OVERWRITE - i.e. a cache.put()") { - val tableName = "evict_partitions_with_put" + test("LRU: RDD eviction accounts for INSERT OVERWRITE - a cache.put()") { + val tableName = "evict_partitions_insert_overwrite" val partitionedTable = createCachedPartitionedTable( tableName, 3 /* numPartitionsToCreate */, @@ -588,9 +588,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert overwrite table evict_partitions_with_put partition(keypart = 1) + sc.runSql("""insert overwrite table evict_partitions_insert_overwrite partition(keypart = 1) select * from test""") - sc.runSql("""insert into table evict_partitions_with_put partition(keypart = 4) + sc.runSql("""insert into table evict_partitions_insert_overwrite partition(keypart = 4) select * from test""") assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") @@ -600,6 +600,23 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(keypart2StorageLevel == StorageLevel.NONE) } + test("LRU: RDD eviction accounts for ALTER TABLE DROP PARTITION - a cache.remove()") { + val tableName = "evict_partitions_removals" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + sc.runSql("alter table evict_partitions_removals drop partition(keypart = 1)") + sc.runSql("""insert into table evict_partitions_removals partition(keypart = 4) + select * from test""") + sc.runSql("""insert into table evict_partitions_removals partition(keypart = 5) + select * from test""") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) + } + test("LRU: get() reloads an RDD previously unpersist()'d.") { val tableName = "reload_evicted_partition" val partitionedTable = createCachedPartitionedTable( From 18c71e9c2f34325d7c2012096e13a45617a69cac Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 12:09:53 -0700 Subject: [PATCH 108/331] Sanity check in MemoryStoreSinkOperator for UnionRDDs used for INSERT INTO. --- .../execution/MemoryStoreSinkOperator.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index ef2d8a5a..01244572 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -24,7 +24,7 @@ import scala.reflect.BeanProperty import org.apache.hadoop.io.Writable -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.storage.StorageLevel import shark.{SharkConfVars, SharkEnv} @@ -98,7 +98,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { } val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned(tableName) - var hasPreviousRDD = false + var hasPreviousRDDForUnion = false if (tachyonWriter != null) { // Put the table in Tachyon. @@ -138,12 +138,12 @@ class MemoryStoreSinkOperator extends TerminalOperator { // If this is an insert, find the existing RDD and create a union of the two, and then // put the union into the metadata tracker. rdd = oldRdd match { - case Some(definedRdd) => rdd.union(oldRdd.get.asInstanceOf[RDD[TablePartition]]) - // The oldRdd can be missing if this is an INSERT into a new Hive-partition. - case None => { - hasPreviousRDD = true - rdd + case Some(definedRdd) => { + hasPreviousRDDForUnion = true + rdd.union(oldRdd.get.asInstanceOf[RDD[TablePartition]]) } + // The oldRdd can be missing if this is an INSERT into a new Hive-partition. + case None => rdd } } // Run a job on the original RDD to force it to go into cache. @@ -153,7 +153,8 @@ class MemoryStoreSinkOperator extends TerminalOperator { if (isHivePartitioned) { SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).foreach{ table => rdd.setName(tableName + "(" + hivePartitionKey + ")") - if (useUnionRDD && !hasPreviousRDD) { + if (useUnionRDD && hasPreviousRDDForUnion) { + assert(rdd.isInstanceOf[UnionRDD[_]]) table.updatePartition(hivePartitionKey, rdd) } else { table.putPartition(hivePartitionKey, rdd) From 9cae8746c4dc0c9eceb7d157ee0c4e12291a09d2 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 18:19:56 -0700 Subject: [PATCH 109/331] SparkDDLTask(Work) => SharkDDLTask(Work) --- src/main/scala/shark/SharkDriver.scala | 4 ++-- .../execution/{SparkDDLTask.scala => SharkDDLTask.scala} | 4 ++-- src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) rename src/main/scala/shark/execution/{SparkDDLTask.scala => SharkDDLTask.scala} (94%) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 01bcbf3f..1869c946 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.util.StringUtils import shark.api.TableRDD import shark.api.QueryExecutionException -import shark.execution.{SparkDDLTask, SparkDDLWork, SharkExplainTask, SharkExplainWork, SparkTask, +import shark.execution.{SharkDDLTask, SharkDDLWork, SharkExplainTask, SharkExplainWork, SparkTask, SparkWork} import shark.memstore2.ColumnarSerDe import shark.parse.{QueryContext, SharkSemanticAnalyzerFactory} @@ -63,7 +63,7 @@ private[shark] object SharkDriver extends LogHelper { // Task factory. Add Shark specific tasks. TaskFactory.taskvec.addAll(Seq( - new TaskFactory.taskTuple(classOf[SparkDDLWork], classOf[SparkDDLTask]), + new TaskFactory.taskTuple(classOf[SharkDDLWork], classOf[SharkDDLTask]), new TaskFactory.taskTuple(classOf[SparkWork], classOf[SparkTask]), new TaskFactory.taskTuple(classOf[SharkExplainWork], classOf[SharkExplainTask]))) diff --git a/src/main/scala/shark/execution/SparkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala similarity index 94% rename from src/main/scala/shark/execution/SparkDDLTask.scala rename to src/main/scala/shark/execution/SharkDDLTask.scala index f54f169f..a7392e43 100644 --- a/src/main/scala/shark/execution/SparkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -29,12 +29,12 @@ import shark.{LogHelper, SharkEnv} import shark.memstore2.{CacheType, MemoryMetadataManager} -private[shark] class SparkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { +private[shark] class SharkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { // Used only for CREATE TABLE. var cacheMode: CacheType.CacheType = _ } -private[shark] class SparkDDLTask extends HiveTask[SparkDDLWork] with Serializable with LogHelper { +private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] with Serializable with LogHelper { override def execute(driverContext: DriverContext): Int = { val hiveMetadataDb = Hive.get(conf) diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 2db2026b..60c778c7 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -9,7 +9,7 @@ import org.apache.hadoop.hive.ql.plan.DDLWork import org.apache.spark.rdd.{UnionRDD, RDD} -import shark.execution.SparkDDLWork +import shark.execution.SharkDDLWork import shark.{LogHelper, SharkEnv} import shark.memstore2.MemoryMetadataManager @@ -44,8 +44,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) assert(ddlWork.isInstanceOf[DDLWork]) val alterTableDesc = ddlWork.asInstanceOf[DDLWork].getAlterTblDesc - val sparkDDLWork = new SparkDDLWork(alterTableDesc) - ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + val sharkDDLWork = new SharkDDLWork(alterTableDesc) + ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } From 79d80a42c00a4008cdf7c9d6f5711fc1746378b8 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 7 Oct 2013 18:24:54 -0700 Subject: [PATCH 110/331] Set Shark version to 0.9.0-SNAPSHOT and use Spark's master branch version, 0.9.0-incubating-SNAPSHOT --- project/SharkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 3cb6d9dc..32154764 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -26,9 +26,9 @@ import scala.util.Properties.{ envOrNone => env } object SharkBuild extends Build { // Shark version - val SHARK_VERSION = "0.8.0-SNAPSHOT" + val SHARK_VERSION = "0.9.0-SNAPSHOT" - val SPARK_VERSION = "0.8.0-SNAPSHOT" + val SPARK_VERSION = "0.9.0-incubating-SNAPSHOT" val SCALA_VERSION = "2.9.3" From 590f04b9816f5257a7b60fb736d744c4c829e914 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 8 Oct 2013 13:44:07 -0700 Subject: [PATCH 111/331] Share a broadcasted Configuration for scans of Hive-partitioned tables stored on disk. --- .../shark/execution/TableScanOperator.scala | 109 +++++++++++------- 1 file changed, 66 insertions(+), 43 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 27247503..6944b2cb 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -19,6 +19,7 @@ package shark.execution import java.util.{ArrayList, Arrays} import scala.reflect.BeanProperty +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS @@ -32,7 +33,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectIns import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.io.Writable +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{PartitionPruningRDD, RDD, UnionRDD} +import org.apache.spark.SerializableWritable import shark.{SharkConfVars, SharkEnv, Utils} import shark.api.QueryExecutionException @@ -214,15 +217,53 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO * Create a RDD representing the table (with or without partitions). */ override def preprocessRdd(rdd: RDD[_]): RDD[_] = { + // copyTableJobPropertiesToConf and pushFilters both take in JobConfs. + val localJobConf = new JobConf(localHconf) + if (tableDesc != null) { + Utilities.copyTableJobPropertiesToConf(tableDesc, localJobConf) + } + val bufferSize = System.getProperty("spark.buffer.size", "65536") + localJobConf.set("io.file.buffer.size", bufferSize) + + new HiveInputFormat() { + def doPushFilters() { + pushFilters(localJobConf, hiveOp) + } + }.doPushFilters() + + // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate + // from Spark's master all the way to Spark's slaves. + var s3varsSet = false + val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", + "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => + if (localJobConf.get(variableName) != null) { + s3varsSet = true + localJobConf.set(variableName, localJobConf.get(variableName)) + } + } + + // If none of the s3 credentials are set in Hive conf, try use the environmental + // variables for credentials. + if (!s3varsSet) { + Utils.setAwsCredentials(localJobConf) + } + + // Choose the minimum number of splits. If mapred.map.tasks is set, use that unless + // it is smaller than what Spark suggests. + val minSplitsPerRDD = math.max( + localJobConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) + + val broadcastedJobConf = SharkEnv.sc.broadcast(new SerializableWritable(localJobConf)) + if (table.isPartitioned) { logDebug("Making %d Hive partitions".format(parts.size)) - makePartitionRDD(rdd) + makeHivePartitionRDDs(broadcastedJobConf, minSplitsPerRDD) } else { val tablePath = table.getPath.toString val ifc = table.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] logDebug("Table input: %s".format(tablePath)) - createHadoopRdd(tablePath, ifc) + createHadoopFileRdd(tablePath, ifc, broadcastedJobConf, minSplitsPerRDD) } } @@ -243,9 +284,12 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO * tables, a data directory is created for each partition corresponding to keys specified using * 'PARTITION BY'. */ - private def makePartitionRDD[T](rdd: RDD[T]): RDD[_] = { + private def makeHivePartitionRDDs[T]( + broadcastedJobConf: Broadcast[SerializableWritable[JobConf]], + minSplitsPerRDD: Int + ): RDD[_] = { val partitions = parts - val rdds = new Array[RDD[Any]](partitions.size) + val hivePartitionRDDs = new Array[RDD[Any]](partitions.size) var i = 0 partitions.foreach { part => @@ -255,10 +299,11 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO val ifc = partition.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - val parts = createHadoopRdd(tablePath, ifc) + val hivePartitionRDD = createHadoopFileRdd( + tablePath, ifc, broadcastedJobConf, minSplitsPerRDD) val serializedHconf = XmlSerializer.serialize(localHconf, localHconf) - val partRDD = parts.mapPartitions { iter => + val hivePartitionRDDWithColValues = hivePartitionRDD.mapPartitions { iter => val hconf = XmlSerializer.deserialize(serializedHconf).asInstanceOf[HiveConf] val deserializer = partDesc.getDeserializerClass().newInstance() deserializer.initialize(hconf, partDesc.getProperties()) @@ -289,53 +334,31 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO rowWithPartArr.asInstanceOf[Object] } } - rdds(i) = partRDD.asInstanceOf[RDD[Any]] + hivePartitionRDDs(i) = hivePartitionRDDWithColValues.asInstanceOf[RDD[Any]] i += 1 } // Even if we don't use any partitions, we still need an empty RDD - if (rdds.size == 0) { + if (hivePartitionRDDs.size == 0) { SharkEnv.sc.makeRDD(Seq[Object]()) } else { - new UnionRDD(rdds(0).context, rdds) + new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) } } - private def createHadoopRdd(path: String, ifc: Class[InputFormat[Writable, Writable]]) + private def createHadoopFileRdd( + path: String, + inputFormatClass: Class[InputFormat[Writable, Writable]], + broadcastedJobConf: Broadcast[SerializableWritable[JobConf]], + minSplits: Int) : RDD[Writable] = { - val conf = new JobConf(localHconf) - if (tableDesc != null) { - Utilities.copyTableJobPropertiesToConf(tableDesc, conf) - } - new HiveInputFormat() { - def doPushFilters() { - pushFilters(conf, hiveOp) - } - }.doPushFilters() - FileInputFormat.setInputPaths(conf, path) - val bufferSize = System.getProperty("spark.buffer.size", "65536") - conf.set("io.file.buffer.size", bufferSize) - - // Set s3/s3n credentials. Setting them in conf ensures the settings propagate - // from Spark's master all the way to Spark's slaves. - var s3varsSet = false - val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", - "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => - if (localHconf.get(variableName) != null) { - s3varsSet = true - conf.set(variableName, localHconf.get(variableName)) - } - } - - // If none of the s3 credentials are set in Hive conf, try use the environmental - // variables for credentials. - if (!s3varsSet) { - Utils.setAwsCredentials(conf) - } - // Choose the minimum number of splits. If mapred.map.tasks is set, use that unless - // it is smaller than what Spark suggests. - val minSplits = math.max(localHconf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) - val rdd = SharkEnv.sc.hadoopRDD(conf, ifc, classOf[Writable], classOf[Writable], minSplits) + val rdd = SharkEnv.sc.hadoopFile( + path, + broadcastedJobConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], + inputFormatClass, + classOf[Writable], + classOf[Writable], + minSplits) // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) From b0d8fadf2620c822408c841ac6a449d73bcbce34 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 8 Oct 2013 23:49:22 -0700 Subject: [PATCH 112/331] Finish merging with master branch. --- .../shark/parse/SharkDDLSemanticAnalyzer.scala | 14 +++++++------- .../scala/shark/parse/SharkSemanticAnalyzer.scala | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 1efa45af..89c93b4c 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -9,7 +9,7 @@ import org.apache.hadoop.hive.ql.plan.DDLWork import org.apache.spark.rdd.{UnionRDD, RDD} -import shark.execution.SparkDDLWork +import shark.execution.SharkDDLWork import shark.{LogHelper, SharkEnv} import shark.memstore2.MemoryMetadataManager @@ -48,8 +48,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) // and DDLWorks that contain AddPartitionDesc objects. for (ddlTask <- rootTasks) { val addPartitionDesc = ddlTask.getWork.asInstanceOf[DDLWork].getAddPartitionDesc - val sparkDDLWork = new SparkDDLWork(addPartitionDesc) - ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + val sharkDDLWork = new SharkDDLWork(addPartitionDesc) + ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } } @@ -62,8 +62,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) // and DDLWorks that contain AddPartitionDesc objects. for (ddlTask <- rootTasks) { val dropTableDesc = ddlTask.getWork.asInstanceOf[DDLWork].getDropTblDesc - val sparkDDLWork = new SparkDDLWork(dropTableDesc) - ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + val sharkDDLWork = new SharkDDLWork(dropTableDesc) + ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } } @@ -82,8 +82,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) assert(ddlWork.isInstanceOf[DDLWork]) val alterTableDesc = ddlWork.asInstanceOf[DDLWork].getAlterTblDesc - val sparkDDLWork = new SparkDDLWork(alterTableDesc) - ddlTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + val sharkDDLWork = new SharkDDLWork(alterTableDesc) + ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 316749ce..ec1ba97a 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -40,7 +40,7 @@ import org.apache.spark.storage.StorageLevel import shark.{CachedTableRecovery, LogHelper, SharkConfVars, SharkEnv, Utils} import shark.execution.{HiveOperator, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator, - SparkDDLWork, SparkWork, TerminalOperator} + SharkDDLWork, SparkWork, TerminalOperator} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} @@ -413,9 +413,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // CREATE TABLE is valid. So, hook a SharkDDLTask as a dependent of the Hive DDLTask so that // Shark metadata is updated only if the Hive task execution is successful. val hiveDDLTask = ddlTasks.head; - val sparkDDLWork = new SparkDDLWork(createTableDesc) - sparkDDLWork.cacheMode = cacheMode - hiveDDLTask.addDependentTask(TaskFactory.get(sparkDDLWork, conf)) + val sharkDDLWork = new SharkDDLWork(createTableDesc) + sharkDDLWork.cacheMode = cacheMode + hiveDDLTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } queryBlock.setCacheModeForCreateTable(cacheMode) From 7e8c7f6b60f00e66c951d039865a6eefd97d1110 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 9 Oct 2013 00:57:23 -0700 Subject: [PATCH 113/331] Comments for SharkEnv, SharkConfVars, MemoryStoreSinkOperator. --- src/main/scala/shark/SharkConfVars.scala | 9 +- src/main/scala/shark/SharkEnv.scala | 7 + .../execution/MemoryStoreSinkOperator.scala | 137 ++++++++++++------ 3 files changed, 104 insertions(+), 49 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index b3eb400b..5672ebea 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -43,14 +43,17 @@ object SharkConfVars { // Default storage level for cached tables. val STORAGE_LEVEL = new ConfVar("shark.cache.storageLevel", "MEMORY_AND_DISK") - // Class name of the default cache policy for cached, Hive-partitioned tables. + // Class name of the default cache policy used to manage partition evictions for cached, + // Hive-partitioned tables. val CACHE_POLICY = new ConfVar( "shark.cache.partition.cachePolicy.class", "shark.memstore2.LRUCachePolicy") - // Max size of the cache specified by the "shark.cache.partition.cachePolicy" property. + // Maximum size - in terms of the number of objects - of the cache specified by the + // "shark.cache.partition.cachePolicy" property above. val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.partition.cachePolicy.maxSize", "10") - // Whether the cache specified by "shark.cache.partition.cachePolicy" should record statistics. + // Whether the cache specified by "shark.cache.partition.cachePolicy" above should record + // statistics (e.g., hit rate). val SHOULD_RECORD_PARTITION_CACHE_STATS = new ConfVar( "shark.cache.partition.cachePolicy.shouldRecordStats", false) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index e067b243..02210ec3 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -114,6 +114,13 @@ object SharkEnv extends LogHelper { val addedFiles = HashSet[String]() val addedJars = HashSet[String]() + /** + * Removes the table associated with 'key'. This method checks for Tachyon tables before + * delegating to MemoryMetadataManager#unpersist() for the actual functionality. + * + * @param key The table that should be dropped from the Shark metastore and removed from memory + * storage. + */ def unpersist(key: String): Option[RDD[_]] = { if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(key)) { if (SharkEnv.tachyonUtil.dropTable(key)) { diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 01244572..16dd4c93 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -38,13 +38,36 @@ import shark.tachyon.TachyonTableWriter */ class MemoryStoreSinkOperator extends TerminalOperator { + // The initial capacity for ArrayLists used to represent columns in columnarcache. If -1, + // the ColumnarSerde will obtain the partition size from a Configuration during execution + // initialization (see ColumnarSerde#initialize()). @BeanProperty var partitionSize: Int = _ + + // If true, columnar storage will use compression. @BeanProperty var shouldCompress: Boolean = _ + + // Storage level to use for the RDD created andmaterialized by this sink operator. @BeanProperty var storageLevel: StorageLevel = _ + + // For CTAS, this is the name of the table that will be created. For INSERTS, this is the name of + // the table that is modified. @BeanProperty var tableName: String = _ + + // Used only for commands on Hive partitions. This partition key is a set of unique values for the + // the table's partitioning columns, and identifies the partition (represented by an RDD) that + // will be created or modified for an INSERT command. @BeanProperty var hivePartitionKey: String = _ + + // The memory storage used to store the output RDD - e.g., CacheType.HEAP refers to Spark's block + // manager. @transient var cacheMode: CacheType.CacheType = _ + + // Whether to compose a UnionRDD from the output RDD and a previous RDD. For example, for an + // INSERT INTO command, the previous RDD will contain the contents of the 'tableName'. @transient var useUnionRDD: Boolean = _ + + // The number of columns in the schema for the table corresponding to 'tableName'. Used only + // to create a TachyonTableWriter, if Tachyon is used. @transient var numColumns: Int = _ override def initializeOnMaster() { @@ -75,36 +98,49 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Put all rows of the table into a set of TablePartition's. Each partition contains // only one TablePartition object. - var rdd: RDD[TablePartition] = inputRdd.mapPartitionsWithIndex { case(partitionIndex, iter) => - op.initializeOnSlave() - val serde = new ColumnarSerDe - serde.initialize(op.localHconf, op.localHiveOp.getConf.getTableInfo.getProperties) - - // Serialize each row into the builder object. - // ColumnarSerDe will return a TablePartitionBuilder. - var builder: Writable = null - iter.foreach { row => - builder = serde.serialize(row.asInstanceOf[AnyRef], op.objectInspector) - } + var outputRDD: RDD[TablePartition] = inputRdd.mapPartitionsWithIndex { + case(partitionIndex, iter) => { + op.initializeOnSlave() + val serde = new ColumnarSerDe + serde.initialize(op.localHconf, op.localHiveOp.getConf.getTableInfo.getProperties) - if (builder != null) { - statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) - Iterator(builder.asInstanceOf[TablePartitionBuilder].build) - } else { - // Empty partition. - statsAcc += Tuple2(partitionIndex, new TablePartitionStats(Array(), 0)) - Iterator(new TablePartition(0, Array())) + // Serialize each row into the builder object. + // ColumnarSerDe will return a TablePartitionBuilder. + var builder: Writable = null + iter.foreach { row => + builder = serde.serialize(row.asInstanceOf[AnyRef], op.objectInspector) + } + + if (builder != null) { + statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) + Iterator(builder.asInstanceOf[TablePartitionBuilder].build) + } else { + // Empty partition. + statsAcc += Tuple2(partitionIndex, new TablePartitionStats(Array(), 0)) + Iterator(new TablePartition(0, Array())) + } } } val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned(tableName) + if (isHivePartitioned) { + // Sanity check: make sure that the Hive-partitioned table corresponding to 'tableName' + // exists in the MemoryMetadataManager. + assert(SharkEnv.memoryMetadataManager.containsTable(tableName), + """Internal Error: Hive-partitioned table %s does not exist in + SharkEnv.memoryMetadataManager.""".format(tableName)) + } + + // If true, a UnionRDD will be used to combine the RDD that contains the query output with the + // previous RDD, which is fetched using 'tableName' or, if the table is Hive-partitioned, the + // ('tableName', 'hivePartitionKey') pair. var hasPreviousRDDForUnion = false if (tachyonWriter != null) { // Put the table in Tachyon. op.logInfo("Putting RDD for %s in Tachyon".format(tableName)) tachyonWriter.createTable(ByteBuffer.allocate(0)) - rdd = rdd.mapPartitionsWithIndex { case(partitionIndex, iter) => + outputRDD = outputRDD.mapPartitionsWithIndex { case(partitionIndex, iter) => val partition = iter.next() partition.toTachyon.zipWithIndex.foreach { case(buf, column) => tachyonWriter.writeColumnPartition(column, partitionIndex, buf) @@ -112,7 +148,8 @@ class MemoryStoreSinkOperator extends TerminalOperator { Iterator(partition) } // Force evaluate so the data gets put into Tachyon. - rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + outputRDD.context.runJob( + outputRDD, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } else { // Put the table in Spark block manager. op.logInfo("Putting %sRDD for %s in Spark block manager, %s %s %s %s".format( @@ -123,48 +160,54 @@ class MemoryStoreSinkOperator extends TerminalOperator { if (storageLevel.useMemory && storageLevel.useDisk) "and" else "", if (storageLevel.useDisk) "on disk" else "")) - // Force evaluate so the data gets put into Spark block manager. - rdd.persist(storageLevel) + outputRDD.persist(storageLevel) - val origRdd = rdd + var queryOutputRDD = outputRDD if (useUnionRDD) { - var oldRdd: Option[RDD[_]] = + // Handle an INSERT INTO command. + var previousRDDOpt: Option[RDD[_]] = if (isHivePartitioned) { - SharkEnv.memoryMetadataManager.getPartitionedTable(tableName) - .flatMap(_.getPartition(hivePartitionKey)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + partitionedTable.getPartition(hivePartitionKey) } else { SharkEnv.memoryMetadataManager.getMemoryTable(tableName).map(_.tableRDD) } - // If this is an insert, find the existing RDD and create a union of the two, and then - // put the union into the metadata tracker. - rdd = oldRdd match { - case Some(definedRdd) => { + outputRDD = previousRDDOpt match { + case Some(previousRDD) => { + // If the table or a Hive-partition that for the INSERT has already been created, + // take a union of the current data and the SELECT output. hasPreviousRDDForUnion = true - rdd.union(oldRdd.get.asInstanceOf[RDD[TablePartition]]) + queryOutputRDD.union(previousRDD.asInstanceOf[RDD[TablePartition]]) } - // The oldRdd can be missing if this is an INSERT into a new Hive-partition. - case None => rdd + // The 'previousRDDOpt' is None if this is an INSERT into a new Hive-partition. + case None => queryOutputRDD } } - // Run a job on the original RDD to force it to go into cache. - origRdd.context.runJob(origRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + // Run a job on the RDD that contains the query output to force the data into the memory + // store. The statistics will also be collected by 'statsAcc' during job execution. + queryOutputRDD.context.runJob( + queryOutputRDD, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } if (isHivePartitioned) { SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).foreach{ table => - rdd.setName(tableName + "(" + hivePartitionKey + ")") + outputRDD.setName(tableName + "(" + hivePartitionKey + ")") if (useUnionRDD && hasPreviousRDDForUnion) { - assert(rdd.isInstanceOf[UnionRDD[_]]) - table.updatePartition(hivePartitionKey, rdd) + // If the Hive-partition has already created, update that partition's metadata entry in + // the PartitionedMemoryTable 'table'. + assert(outputRDD.isInstanceOf[UnionRDD[_]]) + table.updatePartition(hivePartitionKey, outputRDD) } else { - table.putPartition(hivePartitionKey, rdd) + // This is a new Hive-partition. Add a new metadata entry in the Shark Table. + table.putPartition(hivePartitionKey, outputRDD) } } } else { - val table = SharkEnv.memoryMetadataManager.getMemoryTable(tableName).getOrElse( + outputRDD.setName(tableName) + // Create a new MemoryTable entry if one didn't exist (i.e., this operator is for a CTAS). + val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(tableName).getOrElse( SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode, storageLevel)) - rdd.setName(tableName) - table.tableRDD = rdd + memoryTable.tableRDD = outputRDD } // Report remaining memory. @@ -181,9 +224,12 @@ class MemoryStoreSinkOperator extends TerminalOperator { Utils.memoryBytesToString(remainingMems.map(_._2._1).sum))) */ + // TODO(harvey): Get this to work for Hive-partitioned tables. It should be a simple + // 'tableName' + 'hivePartitionKey' concatentation. Though whether stats should belong in + // memstore2.Table should be considered... val columnStats = - if (useUnionRDD) { - // Combine stats for the two tables being combined. + if (useUnionRDD && hasPreviousRDDForUnion) { + // Combine stats for the two RDDs that were combined into UnionRDD. val numPartitions = statsAcc.value.toMap.size val currentStats = statsAcc.value SharkEnv.memoryMetadataManager.getStats(tableName) match { @@ -212,8 +258,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { } } - // Return the cached RDD. - rdd + return outputRDD } override def processPartition(split: Int, iter: Iterator[_]): Iterator[_] = From d0b119a733ed9ea18b9a43fde73b8faf5e395694 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 9 Oct 2013 01:35:42 -0700 Subject: [PATCH 114/331] Comments for RDDUtils, including UnionRDD persistence policy. --- src/main/scala/shark/execution/RDDUtils.scala | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index c94b67a1..1acbdefe 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -36,18 +36,31 @@ object RDDUtils { def getStorageLevelOfRDD(rdd: RDD[_]): StorageLevel = { rdd match { - case u: UnionRDD[_] => getStorageLevelOfRDDs(u.rdds) + case u: UnionRDD[_] => { + // Find the storage level of a UnionRDD from the storage levels of RDDs that compose it. + // A StorageLevel.NONE is returned if all of those RDDs have StorageLevel.NONE. + // Mutually recursive if any RDD in 'u.rdds' is a UnionRDD. + getStorageLevelOfRDDs(u.rdds) + } case _ => rdd.getStorageLevel } } + /** + * Returns the storage level of a sequence of RDDs, interpreted as the storage level of the first + * RDD in the sequence that persisted in memory or disk. + * + * @param rdds The sequence of RDDs to find the StorageLevel of. + */ def getStorageLevelOfRDDs(rdds: Seq[RDD[_]]): StorageLevel = { rdds.foldLeft(StorageLevel.NONE) { (s, r) => { if (s == StorageLevel.NONE) { + // Mutally recursive if 'r' is a UnionRDD. getStorageLevelOfRDD(r) } else { - s + // Some RDD in 'rdds' is persisted in memory or disk, so return early. + return s } } } @@ -56,8 +69,11 @@ object RDDUtils { def unpersistRDD(rdd: RDD[_]): RDD[_] = { rdd match { case u: UnionRDD[_] => { - // Recursively unpersist() all RDDs that compose the UnionRDD. + // Usually, a UnionRDD will not be persisted to avoid data duplication. u.unpersist() + // unpersist() all parent RDDs that compose the UnionRDD. Don't propagate past the parents, + // since a grandparent of the UnionRDD might have multiple child RDDs (i.e., the sibling of + // the UnionRDD's parent is persisted in memory). u.rdds.map { r => r.unpersist() } From 87b4c4841ad5bedb612c07239a432cca1b7de6a2 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 9 Oct 2013 03:48:27 -0700 Subject: [PATCH 115/331] Comments for metastore additions --- src/main/scala/shark/SharkEnv.scala | 12 +- .../execution/MemoryStoreSinkOperator.scala | 28 ++-- .../scala/shark/execution/SharkDDLTask.scala | 128 ++++++++++++------ .../shark/execution/TableScanOperator.scala | 15 +- .../scala/shark/memstore2/CachePolicy.scala | 8 +- .../shark/memstore2/LRUCachePolicy.scala | 2 + .../memstore2/MemoryMetadataManager.scala | 16 +-- .../scala/shark/memstore2/MemoryTable.scala | 3 + .../memstore2/PartitionedMemoryTable.scala | 62 +++++---- src/main/scala/shark/memstore2/Table.scala | 10 +- src/main/scala/shark/parse/QueryBlock.scala | 4 +- .../parse/SharkDDLSemanticAnalyzer.scala | 30 ++-- 12 files changed, 185 insertions(+), 133 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 02210ec3..e7ec70e1 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -115,13 +115,13 @@ object SharkEnv extends LogHelper { val addedJars = HashSet[String]() /** - * Removes the table associated with 'key'. This method checks for Tachyon tables before - * delegating to MemoryMetadataManager#unpersist() for the actual functionality. + * Drops the table associated with 'key'. This method checks for Tachyon tables before + * delegating to MemoryMetadataManager#removeTable() for removing the table's entry in the + * Shark metastore. * - * @param key The table that should be dropped from the Shark metastore and removed from memory - * storage. + * @param key The table that should be dropped from the Shark metastore and from memory storage. */ - def unpersist(key: String): Option[RDD[_]] = { + def dropTable(key: String): Option[RDD[_]] = { if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(key)) { if (SharkEnv.tachyonUtil.dropTable(key)) { logInfo("Table " + key + " was deleted from Tachyon."); @@ -129,7 +129,7 @@ object SharkEnv extends LogHelper { logWarning("Failed to remove table " + key + " from Tachyon."); } } - return memoryMetadataManager.unpersist(key) + return memoryMetadataManager.removeTable(key) } /** Cleans up and shuts down the Shark environments. */ diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 16dd4c93..97d9b5f2 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -123,13 +123,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { } val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned(tableName) - if (isHivePartitioned) { - // Sanity check: make sure that the Hive-partitioned table corresponding to 'tableName' - // exists in the MemoryMetadataManager. - assert(SharkEnv.memoryMetadataManager.containsTable(tableName), - """Internal Error: Hive-partitioned table %s does not exist in - SharkEnv.memoryMetadataManager.""".format(tableName)) - } // If true, a UnionRDD will be used to combine the RDD that contains the query output with the // previous RDD, which is fetched using 'tableName' or, if the table is Hive-partitioned, the @@ -190,17 +183,16 @@ class MemoryStoreSinkOperator extends TerminalOperator { } if (isHivePartitioned) { - SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).foreach{ table => - outputRDD.setName(tableName + "(" + hivePartitionKey + ")") - if (useUnionRDD && hasPreviousRDDForUnion) { - // If the Hive-partition has already created, update that partition's metadata entry in - // the PartitionedMemoryTable 'table'. - assert(outputRDD.isInstanceOf[UnionRDD[_]]) - table.updatePartition(hivePartitionKey, outputRDD) - } else { - // This is a new Hive-partition. Add a new metadata entry in the Shark Table. - table.putPartition(hivePartitionKey, outputRDD) - } + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + outputRDD.setName(tableName + "(" + hivePartitionKey + ")") + if (useUnionRDD && hasPreviousRDDForUnion) { + // An RDD for the Hive partition already exists, so update its metadata entry in + // 'partitionedTable'. + assert(outputRDD.isInstanceOf[UnionRDD[_]]) + partitionedTable.putPartition(hivePartitionKey, outputRDD, true /* isUpdate */) + } else { + // This is a new Hive-partition. Add a new metadata entry in 'partitionedTable'. + partitionedTable.putPartition(hivePartitionKey, outputRDD) } } else { outputRDD.setName(tableName) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 9d38830e..f331213e 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -17,6 +17,8 @@ package shark.execution +import java.util.{List => JavaList, Map => JavaMap} + import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.{Context, DriverContext} @@ -26,52 +28,68 @@ import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.plan.api.StageType import shark.{LogHelper, SharkConfVars, SharkEnv} -import shark.memstore2.{CacheType, MemoryMetadataManager} +import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} private[shark] class SharkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { + // Used only for CREATE TABLE. var cacheMode: CacheType.CacheType = _ } -private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] with Serializable with LogHelper { +/** + * A task used for Shark-specific metastore operations needed for DDL commands, in addition to the + * metastore updates done by Hive's DDLTask. + * + * Validity checks for DDL commands, such as whether a target table for a CREATE TABLE command + * already exists, is not done by SharkDDLTask. Instead, the SharkDDLTask is meant to be used as + * a dependent task of Hive's DDLTask, which handles all error checking. This way, a SharkDDLTask + * is executed only if the Hive DDLTask is successfully executed - i.e., the DDL statement is a + * valid one. + */ +private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] + with Serializable with LogHelper { override def execute(driverContext: DriverContext): Int = { - val hiveMetadataDb = Hive.get(conf) - work.ddlDesc match { case creatTblDesc: CreateTableDesc => { - createTable(hiveMetadataDb, creatTblDesc, work.cacheMode) + createTable(creatTblDesc, work.cacheMode) } case addPartitionDesc: AddPartitionDesc => { - addPartition(hiveMetadataDb, addPartitionDesc) + addPartition(addPartitionDesc) } case dropTableDesc: DropTableDesc => { - dropTable(hiveMetadataDb, dropTableDesc) + dropTableOrPartition(dropTableDesc) } case alterTableDesc: AlterTableDesc => { - alterTable(hiveMetadataDb, alterTableDesc) + alterTable(alterTableDesc) } case _ => { throw new UnsupportedOperationException( - "Shark does not require a Spark DDL task for: " + work.ddlDesc.getClass.getName) + "Shark does not require a Shark DDL task for: " + work.ddlDesc.getClass.getName) } } - // Hive's task runner expects a '0' return value to indicate success and exceptions on - // failure. + // Hive's task runner expects a '0' return value to indicate success, and an exception + // otherwise return 0 } + /** Handles a CREATE TABLE or CTAS. */ def createTable( - hiveMetadataDb: Hive, createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps + + // The preferred storage level is the user-specified storage level for the Shark table's RDD(s). + // This can be different from the actual RDD storage levels at any point in time, depending on + // the the Spark block manager's RDD eviction policy and, for partitioned tables, the + // Hive-partition RDD eviction policy. val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( tblProps.get("shark.cache.storageLevel")) val isHivePartitioned = (createTblDesc.getPartCols.size > 0) + if (isHivePartitioned) { val cachePolicyStr = tblProps.getOrElse("shark.cache.partition.cachePolicy.class", SharkConfVars.CACHE_POLICY.defaultVal) @@ -80,6 +98,9 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] with Serializab val shouldRecordStats = tblProps.getOrElse( "shark.cache.partition.cachePolicy.shouldRecordStats", SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.defaultBoolVal.toString).toBoolean + + // Add a new PartitionedMemoryTable entry in the Shark metastore. + // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( tableName, cacheMode, @@ -90,49 +111,80 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] with Serializab } else { val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( tableName, cacheMode, preferredStorageLevel) + // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. newTable.tableRDD = new EmptyRDD(SharkEnv.sc) } } - def addPartition( - hiveMetadataDb: Hive, - addPartitionDesc: AddPartitionDesc) { + /** Handles an ALTER TABLE ADD PARTITION. */ + def addPartition(addPartitionDesc: AddPartitionDesc) { val tableName = addPartitionDesc.getTableName + val partitionedTable = getPartitionedTableWithAssertions(tableName) + + // Find the set of partition column values that specifies the partition being added. val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionColumns = hiveTable.getPartCols.map(_.getName) - val partitionColumnToValue = addPartitionDesc.getPartSpec - val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( + val partitionColumns: Seq[String] = hiveTable.getPartCols.map(_.getName) + val partitionColumnToValue: JavaMap[String, String] = addPartitionDesc.getPartSpec + // String format for partition key: 'col1=value1/col2=value2/...' + val partitionKeyStr: String = MemoryMetadataManager.makeHivePartitionKeyStr( partitionColumns, partitionColumnToValue) - val partitionedTableOpt = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName) - partitionedTableOpt.map(_.putPartition(keyStr, new EmptyRDD(SharkEnv.sc))) + partitionedTable.putPartition(partitionKeyStr, new EmptyRDD(SharkEnv.sc)) } - def dropTable( - hiveMetadataDb: Hive, - dropTableDesc: DropTableDesc) { + /** + * A DropTableDesc is used for both dropping entire tables (i.e., DROP TABLE) and for dropping + * individual partitions of a table (i.e., ALTER TABLE DROP PARTITION). + */ + def dropTableOrPartition(dropTableDesc: DropTableDesc) { val tableName = dropTableDesc.getTableName + val partitionedTable = getPartitionedTableWithAssertions(tableName) val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionColumns = hiveTable.getPartCols.map(_.getName) - val partSpecs = dropTableDesc.getPartSpecs - for (partSpec <- partSpecs) { - val partitionColumnToValue = partSpec.getPartSpecWithoutOperator - val keyStr = MemoryMetadataManager.makeHivePartitionKeyStr( - partitionColumns, partitionColumnToValue) - val partitionedTableOpt = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName) - partitionedTableOpt.map(_.removePartition(keyStr)) + val partitionSpecs: JavaList[PartitionSpec] = dropTableDesc.getPartSpecs + + if (partitionSpecs == null) { + // The command is a true DROP TABLE. + SharkEnv.dropTable(tableName) + } else { + // The command is an ALTER TABLE DROP PARTITION + // Find the set of partition column values that specifies the partition being dropped. + val partitionColumns: Seq[String] = hiveTable.getPartCols.map(_.getName) + for (partitionSpec <- partitionSpecs) { + val partitionColumnToValue: JavaMap[String, String] = + partitionSpec.getPartSpecWithoutOperator + // String format for partition key: 'col1=value1/col2=value2/...' + val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr( + partitionColumns, partitionColumnToValue) + partitionedTable.removePartition(partitionKeyStr) + } } } - def alterTable( - hiveMetadataDb: Hive, - alterTableDesc: AlterTableDesc) { - if (alterTableDesc.getOp() == AlterTableDesc.AlterTableTypes.RENAME) { - val oldName = alterTableDesc.getOldName - val newName = alterTableDesc.getNewName - SharkEnv.memoryMetadataManager.renameTable(oldName, newName) + /** Handles miscellaneous ALTER TABLE 'tableName' commands. */ + def alterTable(alterTableDesc: AlterTableDesc) { + alterTableDesc.getOp() match { + case AlterTableDesc.AlterTableTypes.RENAME => { + val oldName = alterTableDesc.getOldName + val newName = alterTableDesc.getNewName + SharkEnv.memoryMetadataManager.renameTable(oldName, newName) + } + case _ => { + // TODO(harvey): Support more ALTER TABLE commands, such as ALTER TABLE PARTITION RENAME TO. + throw new UnsupportedOperationException( + "Shark only requires a Shark DDL task for ALTER TABLE RENAME") + } } } + def getPartitionedTableWithAssertions(tableName: String): PartitionedMemoryTable = { + // Sanity checks: make sure that the table we're modifying exists in the Shark metastore and + // is actually partitioned. + val tableOpt = SharkEnv.memoryMetadataManager.getTable(tableName) + assert(tableOpt.isDefined, "Internal Error: table %s doesn't exist in Shark metastore.") + assert(tableOpt.get.isInstanceOf[PartitionedMemoryTable], + "Internal Error: table %s exists in the Shark metastore, but isn't partitioned.") + return tableOpt.get.asInstanceOf[PartitionedMemoryTable] + } + override def getType = StageType.DDL override def getName = "DDL-SPARK" diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 43bdac64..a906e428 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -246,6 +246,14 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } } + /** + * Fetch an RDD from the Shark metastore using each partition key given, and return a union of all + * the fetched RDDs. + * + * @param tableKey Name of the partitioned table. + * @param partitions A collection of Hive-partition metadata, such as partition columns and + * partition key specifications. + */ private def makeCachedPartitionRDD(tableKey: String, partitions: Array[Partition]): RDD[_] = { val hivePartitionRDDSeq = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) @@ -265,10 +273,9 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO new String(partSpec.get(key)) } }.toArray - val partKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partColumns, partSpec) + val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partColumns, partSpec) val hivePartitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableKey).get - val hivePartitionRDD = hivePartitionedTable.getPartition(partKeyStr) - val serializedHconf = XmlSerializer.serialize(localHconf, localHconf) + val hivePartitionRDD = hivePartitionedTable.getPartition(partitionKeyStr) hivePartitionRDD.get.mapPartitions { iter => if (iter.hasNext) { @@ -293,7 +300,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } /** - * Create an RDD for every partition column specified in the query. Note that for on-disk Hive + * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive * tables, a data directory is created for each partition corresponding to keys specified using * 'PARTITION BY'. */ diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index b78a500c..b3ebb4e1 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -27,10 +27,10 @@ trait CachePolicy[K, V] { protected var evictionFunc: (K, V) => Unit = _ def initialize( - maxSize: Long, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit, - shouldRecordStats: Boolean + maxSize: Long, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit, + shouldRecordStats: Boolean ): Unit = { this.maxSize = maxSize this.loadFunc = loadFunc diff --git a/src/main/scala/shark/memstore2/LRUCachePolicy.scala b/src/main/scala/shark/memstore2/LRUCachePolicy.scala index 8c8675b3..62f30acc 100644 --- a/src/main/scala/shark/memstore2/LRUCachePolicy.scala +++ b/src/main/scala/shark/memstore2/LRUCachePolicy.scala @@ -43,12 +43,14 @@ class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { hasRecordedStats = true } + // Executed by 'cache' on the entry being evicted. val removalListener = new RemovalListener[K, V] { def onRemoval(removal: RemovalNotification[K, V]): Unit = { evictionFunc(removal.getKey, removal.getValue) } } + // Executed by 'cache' on the entry being loaded. val cacheLoader = new CacheLoader[K, V] { def load(key: K): V = loadFunc(key) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 09433181..6ca0b433 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -36,7 +36,7 @@ class MemoryMetadataManager { private val _keyToTable: ConcurrentMap[String, Table] = new ConcurrentHashMap[String, Table]() - // TODO(harvey): Support stats for cached Hive-partitioned tables. + // TODO(harvey): Support stats for Hive-partitioned tables. private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] @@ -116,13 +116,6 @@ class MemoryMetadataManager { } } - /** - * Find all keys that are strings. Used to drop tables after exiting. - */ - def getAllKeyStrings(): Seq[String] = { - _keyToTable.keys.collect { case k: String => k } toSeq - } - /** * Used to drop a table from the Spark in-memory cache and/or disk. All metadata * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) tracked by Shark is deleted @@ -133,7 +126,7 @@ class MemoryMetadataManager { * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ - def unpersist(tableName: String): Option[RDD[_]] = { + def removeTable(tableName: String): Option[RDD[_]] = { val lowerCaseTableName = tableName.toLowerCase // Remove MemoryTable's entry from Shark metadata. @@ -142,6 +135,11 @@ class MemoryMetadataManager { val tableValue: Option[Table] = _keyToTable.remove(lowerCaseTableName) return tableValue.flatMap(MemoryMetadataManager.unpersistTable(_)) } + + /** Find all keys that are strings. Used to drop tables after exiting. */ + def getAllKeyStrings(): Seq[String] = { + _keyToTable.keys.collect { case k: String => k } toSeq + } } diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 950e3853..d8849623 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -23,6 +23,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +/** + * A wrapper around a single RDD that backs a Shark table. + */ private[shark] class MemoryTable( tableName: String, diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 83a2dc5d..f35f3a60 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -28,6 +28,13 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +/** + * A container for RDDs that back a Hive-partitioned table. + * + * Note that a Hive-partition of a table is different from an RDD partition. Each Hive-partition + * is stored as a subdirectory of the table subdirectory in the warehouse directory + * (e.g. '/user/hive/warehouse'). So, every Hive-Partition is loaded into Shark as an RDD. + */ private[shark] class PartitionedMemoryTable( tableName: String, @@ -35,6 +42,10 @@ class PartitionedMemoryTable( preferredStorageLevel: StorageLevel) extends Table(tableName, cacheMode, preferredStorageLevel) { + /** + * A simple, mutable wrapper around an RDD. The value entires for a single key in + * '_keyToPartitions' and '_cachePolicy' will reference the same RDDValue object. + */ private class RDDValue(var rdd: RDD[_]) // A map from the Hive-partition key to the RDD that contains contents of that partition. @@ -46,35 +57,31 @@ class PartitionedMemoryTable( // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. private var _cachePolicy: CachePolicy[String, RDDValue] = _ - private var _cachePolicyName: String = "None" - def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) def getPartition(partitionKey: String): Option[RDD[_]] = { - val rddValueFound = _keyToPartitions.get(partitionKey) - if (rddValueFound.isDefined) _cachePolicy.notifyGet(partitionKey) - return rddValueFound.map(_.rdd) + val rddValueOpt: Option[RDDValue] = _keyToPartitions.get(partitionKey) + if (rddValueOpt.isDefined) _cachePolicy.notifyGet(partitionKey) + return rddValueOpt.map(_.rdd) } - def updatePartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { - val rddValueFound = _keyToPartitions.get(partitionKey) - var oldRDD: Option[RDD[_]] = None - if (rddValueFound.isDefined) { - val reusedRDDValue = rddValueFound.get - oldRDD = Some(reusedRDDValue.rdd) - reusedRDDValue.rdd = newRDD - reusedRDDValue + def putPartition( + partitionKey: String, + newRDD: RDD[_], + isUpdate: Boolean = false): Option[RDD[_]] = { + val rddValueOpt = _keyToPartitions.get(partitionKey) + var prevRDD: Option[RDD[_]] = rddValueOpt.map(_.rdd) + if (isUpdate && rddValueOpt.isDefined) { + // This is an update of an old value, so update the RDDValue's 'rdd' entry. + val updatedRDDValue = rddValueOpt.get + updatedRDDValue.rdd = newRDD + updatedRDDValue + } else { + val newRDDValue = new RDDValue(newRDD) + _keyToPartitions.put(partitionKey, newRDDValue) + _cachePolicy.notifyPut(partitionKey, newRDDValue) } - return oldRDD - } - - def putPartition(partitionKey: String, newRDD: RDD[_]): Option[RDD[_]] = { - val rddValueFound = _keyToPartitions.get(partitionKey) - var oldRDD: Option[RDD[_]] = rddValueFound.map(_.rdd) - val newRDDValue = new RDDValue(newRDD) - _keyToPartitions.put(partitionKey, newRDDValue) - _cachePolicy.notifyPut(partitionKey, newRDDValue) - return oldRDD + return prevRDD } def removePartition(partitionKey: String): Option[RDD[_]] = { @@ -88,8 +95,9 @@ class PartitionedMemoryTable( maxSize: Long, shouldRecordStats: Boolean ) { - _cachePolicy = - Class.forName(cachePolicyStr).newInstance.asInstanceOf[CachePolicy[String, RDDValue]] + _cachePolicy = Class.forName(cachePolicyStr).newInstance + .asInstanceOf[CachePolicy[String, RDDValue]] + // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. val loadFunc: String => RDDValue = (partitionKey: String) => { val rddValue = _keyToPartitions.get(partitionKey).get @@ -98,14 +106,12 @@ class PartitionedMemoryTable( } rddValue } + // The evitionFunc will unpersist the RDD. val evictionFunc: (String, RDDValue) => Unit = (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) - _cachePolicyName = cachePolicyStr } - def cachePolicyName: String = _cachePolicyName - def cachePolicy: CachePolicy[String, _] = _cachePolicy def keyToPartitions: collection.immutable.Map[String, RDD[_]] = { diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 9a38a733..fc9c0ac4 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -21,17 +21,13 @@ import org.apache.spark.storage.StorageLevel /** - * A container for table metadata specific to Shark and Spark. Currently, this is a lightweight - * wrapper around either an RDD or multiple RDDs if the Shark table is Hive-partitioned. - * - * Note that a Hive-partition of a table is different from an RDD partition. Each Hive-partition - * is stored as a subdirectory of the table subdirectory in the warehouse directory - * (e.g. '/user/hive/warehouse'). So, every Hive-Partition is loaded into Shark as an RDD. + * A container for table metadata specific to Shark and Spark. */ private[shark] abstract class Table( var tableName: String, var cacheMode: CacheType.CacheType, - var preferredStorageLevel: StorageLevel) { + var preferredStorageLevel: StorageLevel + ) { def getPreferredStorageLevel: StorageLevel diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 9f78fec8..16647605 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -24,8 +24,8 @@ import shark.memstore2.CacheType._ /** - * A container for flags and table metadata that's used in SharkSemanticAnalyzer while parsing - * and analyzing command ASTs (e.g. in analyzeCreateTable()). + * A container for flags and table metadata. Used in SharkSemanticAnalyzer while parsing + * and analyzing ASTs (e.g. in SharkSemanticAnalyzer#analyzeCreateTable()). */ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) extends HiveQueryBlock(outerID, alias, isSubQuery) { diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 89c93b4c..5a113fb5 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -21,18 +21,14 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) ast.getToken.getType match { case HiveParser.TOK_DROPTABLE => { - // TODO(harvey): Execute this in SparkDDLTask. This somewhat works right now because - // unpersist() returns silently when the table doesn't exist. However, it - // ignores any drop protections. - SharkEnv.unpersist(getTableName(ast)) + analyzeDropTableOrDropParts(ast) + } + case HiveParser.TOK_ALTERTABLE_DROPPARTS => { + analyzeDropTableOrDropParts(ast) } - // Handle ALTER TABLE for cached, Hive-partitioned tables case HiveParser.TOK_ALTERTABLE_ADDPARTS => { analyzeAlterTableAddParts(ast) } - case HiveParser.TOK_ALTERTABLE_DROPPARTS => { - alterTableDropParts(ast) - } case HiveParser.TOK_ALTERTABLE_RENAME => { analyzeAlterTableRename(ast) } @@ -40,29 +36,29 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) } } - def analyzeAlterTableAddParts(ast: ASTNode) { + def analyzeDropTableOrDropParts(ast: ASTNode) { val tableName = getTableName(ast) - // Create a SparkDDLTask only if the table is cached. + // Create a SharkDDLTask only if the table is cached. if (SharkEnv.memoryMetadataManager.containsTable(tableName)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks - // and DDLWorks that contain AddPartitionDesc objects. + // and DDLWorks that contain DropTableDesc objects. for (ddlTask <- rootTasks) { - val addPartitionDesc = ddlTask.getWork.asInstanceOf[DDLWork].getAddPartitionDesc - val sharkDDLWork = new SharkDDLWork(addPartitionDesc) + val dropTableDesc = ddlTask.getWork.asInstanceOf[DDLWork].getDropTblDesc + val sharkDDLWork = new SharkDDLWork(dropTableDesc) ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } } - def alterTableDropParts(ast: ASTNode) { + def analyzeAlterTableAddParts(ast: ASTNode) { val tableName = getTableName(ast) - // Create a SparkDDLTask only if the table is cached. + // Create a SharkDDLTask only if the table is cached. if (SharkEnv.memoryMetadataManager.containsTable(tableName)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks // and DDLWorks that contain AddPartitionDesc objects. for (ddlTask <- rootTasks) { - val dropTableDesc = ddlTask.getWork.asInstanceOf[DDLWork].getDropTblDesc - val sharkDDLWork = new SharkDDLWork(dropTableDesc) + val addPartitionDesc = ddlTask.getWork.asInstanceOf[DDLWork].getAddPartitionDesc + val sharkDDLWork = new SharkDDLWork(addPartitionDesc) ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } From 6e86796c773520f94788fcd63ab7c3358b7d9e09 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 9 Oct 2013 05:15:29 -0700 Subject: [PATCH 116/331] Make CachePolicy optional in PartitionedMemoryTable. --- src/main/scala/shark/SharkConfVars.scala | 11 ++++++---- .../scala/shark/execution/SharkDDLTask.scala | 17 +++------------- .../shark/memstore2/LRUCachePolicy.scala | 4 ++-- .../memstore2/MemoryMetadataManager.scala | 20 ++++++++++++++----- .../memstore2/PartitionedMemoryTable.scala | 17 +++++++++------- src/test/scala/shark/SQLSuite.scala | 6 +++--- 6 files changed, 40 insertions(+), 35 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 5672ebea..8f502638 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -43,6 +43,9 @@ object SharkConfVars { // Default storage level for cached tables. val STORAGE_LEVEL = new ConfVar("shark.cache.storageLevel", "MEMORY_AND_DISK") + // True if a table's partitions should be evicted using a cache policy. + val SHOULD_USE_CACHE_POLICY = new ConfVar("shark.cache.partition.useCachePolicy", true) + // Class name of the default cache policy used to manage partition evictions for cached, // Hive-partitioned tables. val CACHE_POLICY = new ConfVar( @@ -181,18 +184,18 @@ case class ConfVar( } def this(varname: String, defaultVal: Int) = { - this(varname, classOf[Int], null, defaultVal, 0, 0, false) + this(varname, classOf[Int], defaultVal.toString, defaultVal, 0, 0, false) } def this(varname: String, defaultVal: Long) = { - this(varname, classOf[Long], null, 0, defaultVal, 0, false) + this(varname, classOf[Long], defaultVal.toString, 0, defaultVal, 0, false) } def this(varname: String, defaultVal: Float) = { - this(varname, classOf[Float], null, 0, 0, defaultVal, false) + this(varname, classOf[Float], defaultVal.toString, 0, 0, defaultVal, false) } def this(varname: String, defaultVal: Boolean) = { - this(varname, classOf[Boolean], null, 0, 0, 0, defaultVal) + this(varname, classOf[Boolean], defaultVal.toString, 0, 0, 0, defaultVal) } } diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index f331213e..9304e7f2 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -89,25 +89,14 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( tblProps.get("shark.cache.storageLevel")) val isHivePartitioned = (createTblDesc.getPartCols.size > 0) - if (isHivePartitioned) { - val cachePolicyStr = tblProps.getOrElse("shark.cache.partition.cachePolicy.class", - SharkConfVars.CACHE_POLICY.defaultVal) - val maxCacheSize = tblProps.getOrElse("shark.cache.partition.cachePolicy.maxSize", - SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toLong - val shouldRecordStats = tblProps.getOrElse( - "shark.cache.partition.cachePolicy.shouldRecordStats", - SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.defaultBoolVal.toString).toBoolean - // Add a new PartitionedMemoryTable entry in the Shark metastore. // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( tableName, cacheMode, preferredStorageLevel, - cachePolicyStr, - maxCacheSize, - shouldRecordStats) + tblProps) } else { val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( tableName, cacheMode, preferredStorageLevel) @@ -137,7 +126,6 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] */ def dropTableOrPartition(dropTableDesc: DropTableDesc) { val tableName = dropTableDesc.getTableName - val partitionedTable = getPartitionedTableWithAssertions(tableName) val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); val partitionSpecs: JavaList[PartitionSpec] = dropTableDesc.getPartSpecs @@ -146,6 +134,7 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] SharkEnv.dropTable(tableName) } else { // The command is an ALTER TABLE DROP PARTITION + val partitionedTable = getPartitionedTableWithAssertions(tableName) // Find the set of partition column values that specifies the partition being dropped. val partitionColumns: Seq[String] = hiveTable.getPartCols.map(_.getName) for (partitionSpec <- partitionSpecs) { @@ -181,7 +170,7 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val tableOpt = SharkEnv.memoryMetadataManager.getTable(tableName) assert(tableOpt.isDefined, "Internal Error: table %s doesn't exist in Shark metastore.") assert(tableOpt.get.isInstanceOf[PartitionedMemoryTable], - "Internal Error: table %s exists in the Shark metastore, but isn't partitioned.") + "Internal Error: table %s isn't partitioned when it should be.") return tableOpt.get.asInstanceOf[PartitionedMemoryTable] } diff --git a/src/main/scala/shark/memstore2/LRUCachePolicy.scala b/src/main/scala/shark/memstore2/LRUCachePolicy.scala index 62f30acc..c0d808f0 100644 --- a/src/main/scala/shark/memstore2/LRUCachePolicy.scala +++ b/src/main/scala/shark/memstore2/LRUCachePolicy.scala @@ -43,14 +43,14 @@ class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { hasRecordedStats = true } - // Executed by 'cache' on the entry being evicted. + // Automatically executed by 'cache' on the entry being evicted. val removalListener = new RemovalListener[K, V] { def onRemoval(removal: RemovalNotification[K, V]): Unit = { evictionFunc(removal.getKey, removal.getValue) } } - // Executed by 'cache' on the entry being loaded. + // Automatically executed by 'cache' on the entry being loaded. val cacheLoader = new CacheLoader[K, V] { def load(key: K): V = loadFunc(key) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 6ca0b433..a12e8479 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -54,13 +54,23 @@ class MemoryMetadataManager { tableName: String, cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, - cachePolicyStr: String, - cachePolicyMaxSize: Long, - shouldRecordStats: Boolean + tblProps: JavaMap[String, String] ): PartitionedMemoryTable = { var newTable = new PartitionedMemoryTable( - tableName.toLowerCase, cacheMode, preferredStorageLevel) - newTable.setPartitionCachePolicy(cachePolicyStr, cachePolicyMaxSize, shouldRecordStats) + tableName.toLowerCase, cacheMode, preferredStorageLevel) + val shouldUseCachePolicy = tblProps.getOrElse( + SharkConfVars.SHOULD_USE_CACHE_POLICY.varname, + SharkConfVars.SHOULD_USE_CACHE_POLICY.defaultBoolVal.toString).toBoolean + if (shouldUseCachePolicy) { + val cachePolicyStr = tblProps.getOrElse(SharkConfVars.CACHE_POLICY.varname, + SharkConfVars.CACHE_POLICY.defaultVal) + val maxCacheSize = tblProps.getOrElse(SharkConfVars.MAX_PARTITION_CACHE_SIZE.varname, + SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toLong + val shouldRecordStats = tblProps.getOrElse( + SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.varname, + SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.defaultVal).toBoolean + newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize, shouldRecordStats) + } _keyToTable.put(tableName.toLowerCase, newTable) return newTable } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index f35f3a60..c14371d3 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -55,13 +55,13 @@ class PartitionedMemoryTable( // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. - private var _cachePolicy: CachePolicy[String, RDDValue] = _ + private var _cachePolicy: Option[CachePolicy[String, RDDValue]] = _ def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) def getPartition(partitionKey: String): Option[RDD[_]] = { val rddValueOpt: Option[RDDValue] = _keyToPartitions.get(partitionKey) - if (rddValueOpt.isDefined) _cachePolicy.notifyGet(partitionKey) + if (rddValueOpt.isDefined && _cachePolicy.isDefined) _cachePolicy.get.notifyGet(partitionKey) return rddValueOpt.map(_.rdd) } @@ -79,14 +79,16 @@ class PartitionedMemoryTable( } else { val newRDDValue = new RDDValue(newRDD) _keyToPartitions.put(partitionKey, newRDDValue) - _cachePolicy.notifyPut(partitionKey, newRDDValue) + if (_cachePolicy.isDefined) _cachePolicy.get.notifyPut(partitionKey, newRDDValue) } return prevRDD } def removePartition(partitionKey: String): Option[RDD[_]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) - if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey, rddRemoved.get) + if (rddRemoved.isDefined && _cachePolicy.isDefined) { + _cachePolicy.get.notifyRemove(partitionKey, rddRemoved.get) + } return rddRemoved.map(_.rdd) } @@ -95,7 +97,7 @@ class PartitionedMemoryTable( maxSize: Long, shouldRecordStats: Boolean ) { - _cachePolicy = Class.forName(cachePolicyStr).newInstance + val newPolicy = Class.forName(cachePolicyStr).newInstance .asInstanceOf[CachePolicy[String, RDDValue]] // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. val loadFunc: String => RDDValue = @@ -109,10 +111,11 @@ class PartitionedMemoryTable( // The evitionFunc will unpersist the RDD. val evictionFunc: (String, RDDValue) => Unit = (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) - _cachePolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) + newPolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) + _cachePolicy = Some(newPolicy) } - def cachePolicy: CachePolicy[String, _] = _cachePolicy + def cachePolicy: Option[CachePolicy[String, _]] = _cachePolicy def keyToPartitions: collection.immutable.Map[String, RDD[_]] = { return _keyToPartitions.mapValues(_.rdd).toMap diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index ca8e3449..f6daa1a1 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -515,7 +515,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { partitioned by (keypart int)""") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val cachePolicy = partitionedTable.cachePolicy + val cachePolicy = partitionedTable.cachePolicy.get assert(cachePolicy.isInstanceOf[shark.memstore2.LRUCachePolicy[_, _]]) } @@ -649,7 +649,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy", true /* shouldRecordStats */) - val lruCachePolicy = partitionedTable.cachePolicy + val lruCachePolicy = partitionedTable.cachePolicy.get val hitRate = lruCachePolicy.getHitRate assert(hitRate.isDefined) assert(hitRate.get == 1.0) @@ -670,7 +670,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val tableName = "dont_record_partition_cache_stats" assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val lruCachePolicy = partitionedTable.cachePolicy + val lruCachePolicy = partitionedTable.cachePolicy.get val hitRate = lruCachePolicy.getHitRate assert(hitRate.isEmpty) val evictionCount = lruCachePolicy.getEvictionCount From fb9e350d31a3f333acc1d820f9bc050047cc7def Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 9 Oct 2013 06:19:14 -0700 Subject: [PATCH 117/331] More comments about metastore additions. --- src/main/scala/shark/SharkEnv.scala | 16 ++--- .../scala/shark/execution/SharkDDLTask.scala | 4 -- .../shark/execution/TableScanOperator.scala | 1 - .../memstore2/MemoryMetadataManager.scala | 58 ++++++++++--------- .../scala/shark/memstore2/MemoryTable.scala | 4 +- .../memstore2/PartitionedMemoryTable.scala | 24 ++++++-- src/main/scala/shark/memstore2/Table.scala | 15 ++++- .../shark/parse/SharkSemanticAnalyzer.scala | 2 +- 8 files changed, 71 insertions(+), 53 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index e7ec70e1..9ff0c8bd 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -116,20 +116,20 @@ object SharkEnv extends LogHelper { /** * Drops the table associated with 'key'. This method checks for Tachyon tables before - * delegating to MemoryMetadataManager#removeTable() for removing the table's entry in the + * delegating to MemoryMetadataManager#removeTable() for removing the table's entry from the * Shark metastore. * - * @param key The table that should be dropped from the Shark metastore and from memory storage. + * @param tableName The table that should be dropped from the Shark metastore and from memory storage. */ - def dropTable(key: String): Option[RDD[_]] = { - if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(key)) { - if (SharkEnv.tachyonUtil.dropTable(key)) { - logInfo("Table " + key + " was deleted from Tachyon."); + def dropTable(tableName: String): Option[RDD[_]] = { + if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(tableName)) { + if (SharkEnv.tachyonUtil.dropTable(tableName)) { + logInfo("Table " + tableName + " was deleted from Tachyon."); } else { - logWarning("Failed to remove table " + key + " from Tachyon."); + logWarning("Failed to remove table " + tableName + " from Tachyon."); } } - return memoryMetadataManager.removeTable(key) + return memoryMetadataManager.removeTable(tableName) } /** Cleans up and shuts down the Shark environments. */ diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 9304e7f2..8d615e4e 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -82,10 +82,6 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps - // The preferred storage level is the user-specified storage level for the Shark table's RDD(s). - // This can be different from the actual RDD storage levels at any point in time, depending on - // the the Spark block manager's RDD eviction policy and, for partitioned tables, the - // Hive-partition RDD eviction policy. val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( tblProps.get("shark.cache.storageLevel")) val isHivePartitioned = (createTblDesc.getPartCols.size > 0) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index a906e428..7e17199e 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -265,7 +265,6 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // Partitioning keys are delimited by "/" val partColumns = partColsDelimited.trim().split("/").toSeq // 'partValues[i]' contains the value for the partitioning column at 'partColumns[i]'. - // TODO(harvey): When is partSpec null? val partValues = partColumns.map { key => if (partSpec == null) { new String diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index a12e8479..082a5167 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -40,6 +40,14 @@ class MemoryMetadataManager { private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] + def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { + _keyToStats.put(key.toLowerCase, stats) + } + + def getStats(key: String): Option[collection.Map[Int, TablePartitionStats]] = { + _keyToStats.get(key.toLowerCase) + } + def createMemoryTable( tableName: String, cacheMode: CacheType.CacheType, @@ -62,15 +70,19 @@ class MemoryMetadataManager { SharkConfVars.SHOULD_USE_CACHE_POLICY.varname, SharkConfVars.SHOULD_USE_CACHE_POLICY.defaultBoolVal.toString).toBoolean if (shouldUseCachePolicy) { - val cachePolicyStr = tblProps.getOrElse(SharkConfVars.CACHE_POLICY.varname, + // Determine the cache policy to use and read any user-specified cache settings. + val cachePolicyStr = tblProps.getOrElse( + SharkConfVars.CACHE_POLICY.varname, SharkConfVars.CACHE_POLICY.defaultVal) - val maxCacheSize = tblProps.getOrElse(SharkConfVars.MAX_PARTITION_CACHE_SIZE.varname, + val maxCacheSize = tblProps.getOrElse( + SharkConfVars.MAX_PARTITION_CACHE_SIZE.varname, SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toLong val shouldRecordStats = tblProps.getOrElse( SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.varname, SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.defaultVal).toBoolean newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize, shouldRecordStats) } + _keyToTable.put(tableName.toLowerCase, newTable) return newTable } @@ -87,29 +99,21 @@ class MemoryMetadataManager { def getTable(tableName: String): Option[Table] = _keyToTable.get(tableName.toLowerCase) def getMemoryTable(tableName: String): Option[MemoryTable] = { - val tableFound = _keyToTable.get(tableName.toLowerCase) - tableFound.foreach(table => - assert(table.isInstanceOf[MemoryTable], - "getMemoryTable() called for a partitioned table.")) - - tableFound.asInstanceOf[Option[MemoryTable]] + val tableOpt = _keyToTable.get(tableName.toLowerCase) + if (tableOpt.isDefined) { + assert(tableOpt.get.isInstanceOf[MemoryTable], + "getMemoryTable() called for a partitioned table.") + } + tableOpt.asInstanceOf[Option[MemoryTable]] } def getPartitionedTable(tableName: String): Option[PartitionedMemoryTable] = { - val tableFound = _keyToTable.get(tableName.toLowerCase) - tableFound.foreach(table => - assert(table.isInstanceOf[PartitionedMemoryTable], - "getPartitionedTable() called for a non-partitioned table.")) - - tableFound.asInstanceOf[Option[PartitionedMemoryTable]] - } - - def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { - _keyToStats.put(key.toLowerCase, stats) - } - - def getStats(key: String): Option[collection.Map[Int, TablePartitionStats]] = { - _keyToStats.get(key.toLowerCase) + val tableOpt = _keyToTable.get(tableName.toLowerCase) + if (tableOpt.isDefined) { + assert(tableOpt.get.isInstanceOf[PartitionedMemoryTable], + "getPartitionedTable() called for a non-partitioned table.") + } + tableOpt.asInstanceOf[Option[PartitionedMemoryTable]] } def renameTable(oldName: String, newName: String) { @@ -127,8 +131,8 @@ class MemoryMetadataManager { } /** - * Used to drop a table from the Spark in-memory cache and/or disk. All metadata - * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) tracked by Shark is deleted + * Used to drop a table from the Spark in-memory cache and/or disk. All metadata tracked by Shark + * tracked by Shark (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) is deleted * as well. * * @param key Name of the table to drop. @@ -143,7 +147,7 @@ class MemoryMetadataManager { _keyToStats.remove(lowerCaseTableName) val tableValue: Option[Table] = _keyToTable.remove(lowerCaseTableName) - return tableValue.flatMap(MemoryMetadataManager.unpersistTable(_)) + return tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) } /** Find all keys that are strings. Used to drop tables after exiting. */ @@ -155,7 +159,7 @@ class MemoryMetadataManager { object MemoryMetadataManager { - def unpersistTable(table: Table): Option[RDD[_]] = { + def unpersistRDDsInTable(table: Table): Option[RDD[_]] = { var unpersistedRDD: Option[RDD[_]] = None if (table.isInstanceOf[PartitionedMemoryTable]) { val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] @@ -172,10 +176,10 @@ object MemoryMetadataManager { return unpersistedRDD } + /** Return a string representation of the partition key, 'col1=value1/col2=value2/...' */ def makeHivePartitionKeyStr( partitionColumns: Seq[String], partitionColumnToValue: JavaMap[String, String]): String = { - // The keyStr is the string 'col1=value1/col2=value2'. var keyStr = "" for (partitionColumn <- partitionColumns) { keyStr += "%s=%s/".format(partitionColumn, partitionColumnToValue(partitionColumn)) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index d8849623..296723e7 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -24,7 +24,7 @@ import org.apache.spark.storage.StorageLevel /** - * A wrapper around a single RDD that backs a Shark table. + * A metadata container for a table in Shark that's backed by an RDD. */ private[shark] class MemoryTable( @@ -40,7 +40,5 @@ class MemoryTable( def tableRDD_= (rdd: RDD[_]) = _tableRDD = rdd - override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel - override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfRDD(tableRDD) } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index c14371d3..0d5cd618 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -29,7 +29,7 @@ import org.apache.spark.storage.StorageLevel /** - * A container for RDDs that back a Hive-partitioned table. + * A metadata container for partitioned Shark table backed by RDDs. * * Note that a Hive-partition of a table is different from an RDD partition. Each Hive-partition * is stored as a subdirectory of the table subdirectory in the warehouse directory @@ -43,19 +43,24 @@ class PartitionedMemoryTable( extends Table(tableName, cacheMode, preferredStorageLevel) { /** - * A simple, mutable wrapper around an RDD. The value entires for a single key in - * '_keyToPartitions' and '_cachePolicy' will reference the same RDDValue object. + * A simple, mutable wrapper for an RDD. The value entires for a single key in + * '_keyToPartitions' and '_cachePolicy' will reference the same RDDValue object. This is needed + * so that entries maintained by a CachePolicy's cache map, such as the LRUCachePolicy#cache map, + * can be updated. The values can't be RDDs, which are inherently immutable... */ private class RDDValue(var rdd: RDD[_]) // A map from the Hive-partition key to the RDD that contains contents of that partition. + // The conventional string format for the partition key, 'col1=value1/col2=value2/...', can be + // computed using MemoryMetadataManager#makeHivePartitionKeyStr() private var _keyToPartitions: ConcurrentMap[String, RDDValue] = new ConcurrentJavaHashMap[String, RDDValue]() // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. - private var _cachePolicy: Option[CachePolicy[String, RDDValue]] = _ + // If 'None', then all partitions will be persisted in memory using the 'preferredStorageLevel'. + private var _cachePolicy: Option[CachePolicy[String, RDDValue]] = None def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) @@ -73,6 +78,9 @@ class PartitionedMemoryTable( var prevRDD: Option[RDD[_]] = rddValueOpt.map(_.rdd) if (isUpdate && rddValueOpt.isDefined) { // This is an update of an old value, so update the RDDValue's 'rdd' entry. + // Don't notify the '_cachePolicy'. Assumes that getPartition() has already been called to + // obtain the value of the previous RDD, and that an RDD update refers to the RDD created from + // a transform or union. val updatedRDDValue = rddValueOpt.get updatedRDDValue.rdd = newRDD updatedRDDValue @@ -117,12 +125,16 @@ class PartitionedMemoryTable( def cachePolicy: Option[CachePolicy[String, _]] = _cachePolicy + /** Returns an immutable view of the String->RDD mapping to external callers */ def keyToPartitions: collection.immutable.Map[String, RDD[_]] = { return _keyToPartitions.mapValues(_.rdd).toMap } - override def getPreferredStorageLevel: StorageLevel = preferredStorageLevel - + /** + * Computes the current storage level for this table. See comments in + * RDDUtils#getStorageLevelOfRDDs() for how Shark interprets the storage level of a sequence of + * RDDs. + */ override def getCurrentStorageLevel: StorageLevel = { return RDDUtils.getStorageLevelOfRDDs(_keyToPartitions.values.map(_.rdd).toSeq) } diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index fc9c0ac4..a6ba4b0d 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -21,7 +21,15 @@ import org.apache.spark.storage.StorageLevel /** - * A container for table metadata specific to Shark and Spark. + * A container for table metadata managed by Shark and Spark. Subclasses are responsible for + * how RDDs are set, stored, and accessed. + * + * @param tableName Name of this table. + * @param cacheMode Type of memory storage used for the table (e.g., the Spark block manager). + * @param preferredStorageLevel The user-specified storage level for the Shark table's RDD(s). + * This can be different from the actual RDD storage levels at any point in time, depending on + * the the Spark block manager's RDD eviction policy and, for partitioned tables, the + * Hive-partition RDD eviction policy. */ private[shark] abstract class Table( var tableName: String, @@ -29,7 +37,8 @@ private[shark] abstract class Table( var preferredStorageLevel: StorageLevel ) { - def getPreferredStorageLevel: StorageLevel - + /** + * Compute the current storage level of RDDs that back this table. + */ def getCurrentStorageLevel: StorageLevel } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index ec1ba97a..b94f2bef 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -178,7 +178,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey(qb) } - val preferredStorageLevel = table.getPreferredStorageLevel + val preferredStorageLevel = table.preferredStorageLevel OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, cachedTableName, From df63897e8b1ebcfce00aace19b9eea86f0f28c88 Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Thu, 10 Oct 2013 23:35:54 -0700 Subject: [PATCH 118/331] Style fixups across all code after the introduction of scalastyle style checker tool. --- .../scala/shark/CachedTableRecovery.scala | 2 +- src/main/scala/shark/SharkConfVars.scala | 3 +- src/main/scala/shark/SharkServer.scala | 6 ++-- src/main/scala/shark/api/ResultSet.scala | 19 ++++++++++- .../scala/shark/execution/CoGroupedRDD.scala | 3 ++ .../shark/execution/CommonJoinOperator.scala | 5 +-- src/main/scala/shark/execution/EmptyRDD.scala | 17 ++++++++++ .../shark/execution/FilterOperator.scala | 2 +- .../execution/GroupByPreShuffleOperator.scala | 9 ++--- .../scala/shark/execution/JoinOperator.scala | 3 +- src/main/scala/shark/execution/JoinUtil.scala | 5 +-- .../shark/execution/MapJoinOperator.scala | 6 ++-- .../shark/execution/MapSplitPruning.scala | 17 ++++++---- .../scala/shark/execution/ReduceKey.scala | 3 +- .../shark/execution/ReduceSinkOperator.scala | 3 +- .../execution/optimization/ColumnPruner.scala | 17 ++++++++++ .../serialization/HiveConfSerializer.scala | 3 +- .../OperatorSerializationWrapper.scala | 2 +- .../serialization/ShuffleSerializer.scala | 3 +- .../ColumnarStructObjectInspector.scala | 3 +- .../shark/memstore2/column/ColumnType.scala | 3 +- .../column/CompressionAlgorithm.scala | 20 ++++++++++- .../column/NullableColumnBuilder.scala | 2 +- .../parse/SharkDDLSemanticAnalyzer.scala | 20 ++++++++++- .../shark/parse/SharkSemanticAnalyzer.scala | 4 +-- .../parse/SharkSemanticAnalyzerFactory.scala | 3 +- src/main/scala/shark/util/BloomFilter.scala | 32 ++++++++++++++---- .../shark/util/MurmurHash3_x86_128.scala | 33 ++++++++++++++----- 28 files changed, 196 insertions(+), 52 deletions(-) diff --git a/src/main/scala/shark/CachedTableRecovery.scala b/src/main/scala/shark/CachedTableRecovery.scala index a5ec5e58..4741cf0f 100644 --- a/src/main/scala/shark/CachedTableRecovery.scala +++ b/src/main/scala/shark/CachedTableRecovery.scala @@ -88,4 +88,4 @@ object CachedTableRecovery extends LogHelper { } }) } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index da2112d7..10d6a689 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -65,7 +65,8 @@ object SharkConfVars { conf.set(EXPLAIN_MODE.varname, EXPLAIN_MODE.defaultVal) } if (conf.get(COLUMN_BUILDER_PARTITION_SIZE.varname) == null) { - conf.setInt(COLUMN_BUILDER_PARTITION_SIZE.varname, COLUMN_BUILDER_PARTITION_SIZE.defaultIntVal) + conf.setInt(COLUMN_BUILDER_PARTITION_SIZE.varname, + COLUMN_BUILDER_PARTITION_SIZE.defaultIntVal) } if (conf.get(COLUMNAR_COMPRESSION.varname) == null) { conf.setBoolean(COLUMNAR_COMPRESSION.varname, COLUMNAR_COMPRESSION.defaultBoolVal) diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index cd076a1d..67e36044 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -271,9 +271,11 @@ class SharkServerHandler extends HiveServerHandler with LogHelper { "" } else { val list: JavaList[String] = fetchN(1) - if (list.isEmpty) + if (list.isEmpty) { "" - else list.get(0) + } else { + list.get(0) + } } } diff --git a/src/main/scala/shark/api/ResultSet.scala b/src/main/scala/shark/api/ResultSet.scala index abd49c99..682d769b 100644 --- a/src/main/scala/shark/api/ResultSet.scala +++ b/src/main/scala/shark/api/ResultSet.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api import java.util.{Arrays, Collections, List => JList} @@ -30,4 +47,4 @@ class ResultSet private[shark](_schema: Array[ColumnDesc], _results: Array[Array _results.map(row => row.mkString("\t")).mkString("\n") } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/execution/CoGroupedRDD.scala b/src/main/scala/shark/execution/CoGroupedRDD.scala index 4b5da641..8bd85c28 100644 --- a/src/main/scala/shark/execution/CoGroupedRDD.scala +++ b/src/main/scala/shark/execution/CoGroupedRDD.scala @@ -49,12 +49,15 @@ case class NarrowCoGroupSplitDep( case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep +// equals not implemented style error +// scalastyle:off class CoGroupPartition(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Partition with Serializable { override val index: Int = idx override def hashCode(): Int = idx } +// scalastyle:on class CoGroupAggregator extends Aggregator[Any, Any, ArrayBuffer[Any]]( diff --git a/src/main/scala/shark/execution/CommonJoinOperator.scala b/src/main/scala/shark/execution/CommonJoinOperator.scala index a081ade5..51f2b134 100755 --- a/src/main/scala/shark/execution/CommonJoinOperator.scala +++ b/src/main/scala/shark/execution/CommonJoinOperator.scala @@ -37,8 +37,9 @@ import org.apache.spark.SparkContext.rddToPairRDDFunctions import shark.SharkConfVars -abstract class CommonJoinOperator[JOINDESCTYPE <: JoinDesc, T <: HiveCommonJoinOperator[JOINDESCTYPE]] - extends NaryOperator[T] { +abstract class CommonJoinOperator[JOINDESCTYPE <: JoinDesc, + T <: HiveCommonJoinOperator[JOINDESCTYPE]] + extends NaryOperator[T] { @BeanProperty var conf: JOINDESCTYPE = _ // Order in which the results should be output. diff --git a/src/main/scala/shark/execution/EmptyRDD.scala b/src/main/scala/shark/execution/EmptyRDD.scala index 534a34bc..101660ec 100644 --- a/src/main/scala/shark/execution/EmptyRDD.scala +++ b/src/main/scala/shark/execution/EmptyRDD.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} diff --git a/src/main/scala/shark/execution/FilterOperator.scala b/src/main/scala/shark/execution/FilterOperator.scala index 2548bb59..a2a472dc 100755 --- a/src/main/scala/shark/execution/FilterOperator.scala +++ b/src/main/scala/shark/execution/FilterOperator.scala @@ -56,4 +56,4 @@ class FilterOperator extends UnaryOperator[HiveFilterOperator] { } } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala index 055f682a..9f6b0a76 100755 --- a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala @@ -101,7 +101,8 @@ class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { val keyois = new JArrayList[ObjectInspector](totalFields) keyObjectInspectors.foreach(keyois.add(_)) - keyObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(keyFieldNames, keyois) + keyObjectInspector = ObjectInspectorFactory. + getStandardStructObjectInspector(keyFieldNames, keyois) keyFactory = new KeyWrapperFactory(keyFields, keyObjectInspectors, currentKeyObjectInspectors) } @@ -148,9 +149,9 @@ class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { } else { logInfo("Mapside hash aggregation enabled") } - logInfo("#hash table="+numRowsHashTbl+" #rows="+ - numRowsInput+" reduction="+numRowsHashTbl.toFloat/numRowsInput+ - " minReduction="+minReductionHashAggr) + logInfo("#hash table=" + numRowsHashTbl + " #rows=" + + numRowsInput + " reduction=" + numRowsHashTbl.toFloat/numRowsInput + + " minReduction=" + minReductionHashAggr) } } diff --git a/src/main/scala/shark/execution/JoinOperator.scala b/src/main/scala/shark/execution/JoinOperator.scala index a641a264..5614f6ae 100755 --- a/src/main/scala/shark/execution/JoinOperator.scala +++ b/src/main/scala/shark/execution/JoinOperator.scala @@ -27,7 +27,8 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{JoinOperator => HiveJoinOperator} import org.apache.hadoop.hive.ql.plan.{JoinDesc, TableDesc} import org.apache.hadoop.hive.serde2.{Deserializer, Serializer, SerDeUtils} -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StandardStructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils +import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector import org.apache.hadoop.io.BytesWritable import org.apache.spark.{CoGroupedRDD, HashPartitioner} diff --git a/src/main/scala/shark/execution/JoinUtil.scala b/src/main/scala/shark/execution/JoinUtil.scala index 2fb22731..40c5630b 100644 --- a/src/main/scala/shark/execution/JoinUtil.scala +++ b/src/main/scala/shark/execution/JoinUtil.scala @@ -22,7 +22,8 @@ import java.util.{List => JavaList} import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector => OI} import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils => OIUtils} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.{ObjectInspectorCopyOption => CopyOption} +import org.apache.hadoop.hive.serde2.objectinspector. + ObjectInspectorUtils.{ObjectInspectorCopyOption => CopyOption} import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector => PrimitiveOI} import org.apache.hadoop.io.BooleanWritable import org.apache.hadoop.io.NullWritable @@ -78,7 +79,7 @@ object JoinUtil { if (noOuterJoin) { a } else { - val n = new Array[AnyRef](size+1) + val n = new Array[AnyRef](size + 1) Array.copy(a, 0, n, 0, size) n(size) = new SerializableWritable(new BooleanWritable(isFiltered)) n diff --git a/src/main/scala/shark/execution/MapJoinOperator.scala b/src/main/scala/shark/execution/MapJoinOperator.scala index 400415b5..fd92ead2 100755 --- a/src/main/scala/shark/execution/MapJoinOperator.scala +++ b/src/main/scala/shark/execution/MapJoinOperator.scala @@ -126,8 +126,9 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato // Collect the RDD and build a hash table. val startCollect = System.currentTimeMillis() val storageLevel = rddForHash.getStorageLevel - if(storageLevel == StorageLevel.NONE) + if (storageLevel == StorageLevel.NONE) { rddForHash.persist(StorageLevel.MEMORY_AND_DISK) + } rddForHash.foreach(_ => Unit) val wrappedRows = rddForHash.partitions.flatMap { part => val blockId = "rdd_%s_%s".format(rddForHash.id, part.index) @@ -138,8 +139,9 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato }) partRows } - if(storageLevel == StorageLevel.NONE) + if(storageLevel == StorageLevel.NONE) { rddForHash.unpersist() + } logDebug("wrappedRows size:" + wrappedRows.size) val collectTime = System.currentTimeMillis() - startCollect diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index f7b1634b..f89b8676 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -70,7 +70,10 @@ object MapSplitPruning { } case _: GenericUDFIn => - testInPredicate(s, e.children(0).asInstanceOf[ExprNodeColumnEvaluator], e.children.drop(1)) + testInPredicate( + s, + e.children(0).asInstanceOf[ExprNodeColumnEvaluator], + e.children.drop(1)) case udf: GenericUDFBaseCompare => testComparisonPredicate(s, udf, e.children(0), e.children(1)) case _ => true @@ -135,19 +138,19 @@ object MapSplitPruning { // Try to get the column evaluator. val columnEval: ExprNodeColumnEvaluator = - if (left.isInstanceOf[ExprNodeColumnEvaluator]) + if (left.isInstanceOf[ExprNodeColumnEvaluator]) { left.asInstanceOf[ExprNodeColumnEvaluator] - else if (right.isInstanceOf[ExprNodeColumnEvaluator]) + } else if (right.isInstanceOf[ExprNodeColumnEvaluator]) { right.asInstanceOf[ExprNodeColumnEvaluator] - else null + } else null // Try to get the constant value. val constEval: ExprNodeConstantEvaluator = - if (left.isInstanceOf[ExprNodeConstantEvaluator]) + if (left.isInstanceOf[ExprNodeConstantEvaluator]) { left.asInstanceOf[ExprNodeConstantEvaluator] - else if (right.isInstanceOf[ExprNodeConstantEvaluator]) + } else if (right.isInstanceOf[ExprNodeConstantEvaluator]) { right.asInstanceOf[ExprNodeConstantEvaluator] - else null + } else null if (columnEval != null && constEval != null) { // We can prune the partition only if it is a predicate of form diff --git a/src/main/scala/shark/execution/ReduceKey.scala b/src/main/scala/shark/execution/ReduceKey.scala index d91df3bc..e913ec87 100755 --- a/src/main/scala/shark/execution/ReduceKey.scala +++ b/src/main/scala/shark/execution/ReduceKey.scala @@ -79,7 +79,8 @@ class ReduceKeyMapSide(var bytesWritable: BytesWritable) extends ReduceKey if (length != other.length) { false } else { - WritableComparator.compareBytes(byteArray, 0, length, other.byteArray, 0, other.length) == 0 + WritableComparator. + compareBytes(byteArray, 0, length, other.byteArray, 0, other.length) == 0 } } case _ => false diff --git a/src/main/scala/shark/execution/ReduceSinkOperator.scala b/src/main/scala/shark/execution/ReduceSinkOperator.scala index 6c01ab47..2c022203 100755 --- a/src/main/scala/shark/execution/ReduceSinkOperator.scala +++ b/src/main/scala/shark/execution/ReduceSinkOperator.scala @@ -92,7 +92,8 @@ class ReduceSinkOperator extends UnaryOperator[HiveReduceSinkOperator] { ois.add(keySer.getObjectInspector) ois.add(valueSer.getObjectInspector) - val outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector(List("KEY","VALUE"), ois) + val outputObjInspector = ObjectInspectorFactory. + getStandardStructObjectInspector(List("KEY","VALUE"), ois) val joinTag = conf.getTag() diff --git a/src/main/scala/shark/execution/optimization/ColumnPruner.scala b/src/main/scala/shark/execution/optimization/ColumnPruner.scala index 4ab62194..f116575e 100644 --- a/src/main/scala/shark/execution/optimization/ColumnPruner.scala +++ b/src/main/scala/shark/execution/optimization/ColumnPruner.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution.optimization import java.util.BitSet diff --git a/src/main/scala/shark/execution/serialization/HiveConfSerializer.scala b/src/main/scala/shark/execution/serialization/HiveConfSerializer.scala index db612e4c..8d9dc4bf 100644 --- a/src/main/scala/shark/execution/serialization/HiveConfSerializer.scala +++ b/src/main/scala/shark/execution/serialization/HiveConfSerializer.scala @@ -47,8 +47,9 @@ object HiveConfSerializer { val auxJars = Text.readString(dis) val conf = new HiveConf conf.readFields(dis) - if(auxJars.equals("").unary_!) + if (auxJars.equals("").unary_!) { conf.setAuxJars(auxJars) + } conf } } diff --git a/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala b/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala index 858ce182..6fc69f21 100644 --- a/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala +++ b/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala @@ -74,4 +74,4 @@ object OperatorSerializationWrapper { wrapper.value = value wrapper } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala index b2a2d014..49346161 100644 --- a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala +++ b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala @@ -22,7 +22,8 @@ import java.nio.ByteBuffer import org.apache.hadoop.io.BytesWritable -import org.apache.spark.serializer.{DeserializationStream, Serializer, SerializerInstance, SerializationStream} +import org.apache.spark.serializer. + {DeserializationStream, Serializer, SerializerInstance, SerializationStream} import shark.execution.{ReduceKey, ReduceKeyReduceSide} diff --git a/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala b/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala index 02f799fe..438823f4 100644 --- a/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala +++ b/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala @@ -60,7 +60,8 @@ object ColumnarStructObjectInspector { for (i <- 0 until columnNames.size) { val typeInfo = columnTypes.get(i) val fieldOI = typeInfo.getCategory match { - case Category.PRIMITIVE => PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( + case Category.PRIMITIVE => + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( typeInfo.asInstanceOf[PrimitiveTypeInfo].getPrimitiveCategory) case _ => LazyFactory.createLazyObjectInspector( typeInfo, serDeParams.getSeparators(), 1, serDeParams.getNullSequence(), diff --git a/src/main/scala/shark/memstore2/column/ColumnType.scala b/src/main/scala/shark/memstore2/column/ColumnType.scala index 3d50d2e6..9ed084bc 100644 --- a/src/main/scala/shark/memstore2/column/ColumnType.scala +++ b/src/main/scala/shark/memstore2/column/ColumnType.scala @@ -22,7 +22,8 @@ import java.sql.Timestamp import org.apache.hadoop.hive.serde2.ByteStream import org.apache.hadoop.hive.serde2.`lazy`.{ByteArrayRef, LazyBinary} -import org.apache.hadoop.hive.serde2.io.{TimestampWritable, ShortWritable, ByteWritable, DoubleWritable} +import org.apache.hadoop.hive.serde2.io. + {TimestampWritable, ShortWritable, ByteWritable, DoubleWritable} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io._ diff --git a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala index 1ebd5943..905d32ee 100644 --- a/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala +++ b/src/main/scala/shark/memstore2/column/CompressionAlgorithm.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column import java.nio.{ByteBuffer, ByteOrder} @@ -307,7 +324,8 @@ class BooleanBitSetCompression extends CompressionAlgorithm { // Booleans are encoded into Longs; in addition, we need one int to store the number of // Booleans contained in the compressed buffer. - override def compressedSize: Int = math.ceil(_uncompressedSize.toFloat / BooleanBitSetCompression.BOOLEANS_PER_LONG).toInt * 8 + 4 + override def compressedSize: Int = + math.ceil(_uncompressedSize.toFloat / BooleanBitSetCompression.BOOLEANS_PER_LONG).toInt * 8 + 4 override def uncompressedSize: Int = _uncompressedSize diff --git a/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala b/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala index 53472c7b..2d79fd87 100644 --- a/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala +++ b/src/main/scala/shark/memstore2/column/NullableColumnBuilder.scala @@ -68,4 +68,4 @@ trait NullableColumnBuilder[T] extends ColumnBuilder[T] { newBuffer.rewind() newBuffer } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 60c778c7..564a3167 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -1,10 +1,28 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.parse import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.TaskFactory -import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, HiveParser} +import org.apache.hadoop.hive.ql.parse. + {ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, HiveParser} import org.apache.hadoop.hive.ql.plan.DDLWork import org.apache.spark.rdd.{UnionRDD, RDD} diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index aa6ea812..e92da3b9 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -107,8 +107,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // There are two cases that will enable caching: // 1) Table name includes "_cached" or "_tachyon". - // 2) The "shark.cache" table property is "true", or the string representation of a supported - // cache mode (heap, Tachyon). + // 2) The "shark.cache" table property is "true", or the string representation of a + // supported cache mode (heap, Tachyon). cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) // Continue planning based on the 'cacheMode' read. if (cacheMode == CacheType.HEAP || diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala index 91215988..5ef6efec 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala @@ -38,10 +38,9 @@ object SharkSemanticAnalyzerFactory { SharkConfVars.getVar(conf, SharkConfVars.EXPLAIN_MODE) == "shark") { new SharkExplainSemanticAnalyzer(conf) } else if (baseSem.isInstanceOf[DDLSemanticAnalyzer]) { - new SharkDDLSemanticAnalyzer(conf) + new SharkDDLSemanticAnalyzer(conf) } else { baseSem } } } - diff --git a/src/main/scala/shark/util/BloomFilter.scala b/src/main/scala/shark/util/BloomFilter.scala index 6a26b9e5..8183a6a9 100644 --- a/src/main/scala/shark/util/BloomFilter.scala +++ b/src/main/scala/shark/util/BloomFilter.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.util import java.util.BitSet @@ -19,7 +36,7 @@ import com.google.common.primitives.Longs * @date 07/07/2013 */ class BloomFilter(numBitsPerElement: Double, expectedSize: Int, numHashes: Int) - extends AnyRef with Serializable{ + extends AnyRef with Serializable{ val SEED = System.getProperty("shark.bloomfilter.seed","1234567890").toInt val bitSetSize = ceil(numBitsPerElement * expectedSize).toInt @@ -98,7 +115,7 @@ class BloomFilter(numBitsPerElement: Double, expectedSize: Int, numHashes: Int) * @param data is the bytes to be hashed. * @param length is the length of the buffer to examine. * @return true with some false positive probability and false if the - * bytes is not contained in the bloom filter. + * bytes is not contained in the bloom filter. */ def contains(data: Array[Byte], len: Int): Boolean = { !hash(data,numHashes, len).exists { @@ -119,14 +136,17 @@ class BloomFilter(numBitsPerElement: Double, expectedSize: Int, numHashes: Int) MurmurHash3_x86_128.hash(data, SEED + i, len, results) a(i) = results(0).abs var j = i + 1 - if (j < n) + if (j < n) { a(j) = results(1).abs + } j += 1 - if (j < n) + if (j < n) { a(j) = results(2).abs + } j += 1 - if (j < n) + if (j < n) { a(j) = results(3).abs + } i += 1 } a @@ -139,4 +159,4 @@ object BloomFilter { def numHashes(fpp: Double, expectedSize: Int) = ceil(-(log(fpp) / log(2))).toInt -} \ No newline at end of file +} diff --git a/src/main/scala/shark/util/MurmurHash3_x86_128.scala b/src/main/scala/shark/util/MurmurHash3_x86_128.scala index 5dcc6068..47e9220e 100644 --- a/src/main/scala/shark/util/MurmurHash3_x86_128.scala +++ b/src/main/scala/shark/util/MurmurHash3_x86_128.scala @@ -1,3 +1,20 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.util import java.lang.Integer.{ rotateLeft => rotl } @@ -109,7 +126,7 @@ object MurmurHash3_x86_128 { * @param seed is the seed for the murmurhash algorithm. * @param length is the length of the buffer to use for hashing. * @param results is the output buffer to store the four ints that are returned, - * should have size atleast 4. + * should have size atleast 4. */ @inline final def hash(data: Array[Byte], seed: Int, length: Int, results: Array[Int]): Unit = { @@ -179,16 +196,16 @@ object MurmurHash3_x86_128 { @inline final def getInt(data: Array[Byte], index: Int, rem: Int): Int = { rem match { case 3 => data(index) << 24 | - (data(index + 1) & 0xFF) << 16 | - (data(index + 2) & 0xFF) << 8 + (data(index + 1) & 0xFF) << 16 | + (data(index + 2) & 0xFF) << 8 case 2 => data(index) << 24 | - (data(index + 1) & 0xFF) << 16 + (data(index + 1) & 0xFF) << 16 case 1 => data(index) << 24 case 0 => 0 case _ => data(index) << 24 | - (data(index + 1) & 0xFF) << 16 | - (data(index + 2) & 0xFF) << 8 | - (data(index + 3) & 0xFF) + (data(index + 1) & 0xFF) << 16 | + (data(index + 2) & 0xFF) << 8 | + (data(index + 3) & 0xFF) } } -} \ No newline at end of file +} From d885618ffc738c9a1cf93f47e6061590f830ff55 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 12 Oct 2013 22:50:37 -0700 Subject: [PATCH 119/331] Make all memstore RDD references RDD[TablePartition]. --- src/main/scala/shark/memstore2/MemoryTable.scala | 6 +++--- .../shark/memstore2/PartitionedMemoryTable.scala | 16 ++++++++-------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 296723e7..1fb50ce0 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -34,11 +34,11 @@ class MemoryTable( extends Table(tableName, cacheMode, preferredStorageLevel) { // RDD that contains the contents of this table. - private var _tableRDD: RDD[_] = _ + private var _tableRDD: RDD[TablePartition] = _ - def tableRDD: RDD[_] = _tableRDD + def tableRDD: RDD[TablePartition] = _tableRDD - def tableRDD_= (rdd: RDD[_]) = _tableRDD = rdd + def tableRDD_= (rdd: RDD[TablePartition]) = _tableRDD = rdd override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfRDD(tableRDD) } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 0d5cd618..57c29761 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -48,7 +48,7 @@ class PartitionedMemoryTable( * so that entries maintained by a CachePolicy's cache map, such as the LRUCachePolicy#cache map, * can be updated. The values can't be RDDs, which are inherently immutable... */ - private class RDDValue(var rdd: RDD[_]) + class RDDValue(var rdd: RDD[TablePartition]) // A map from the Hive-partition key to the RDD that contains contents of that partition. // The conventional string format for the partition key, 'col1=value1/col2=value2/...', can be @@ -64,7 +64,7 @@ class PartitionedMemoryTable( def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) - def getPartition(partitionKey: String): Option[RDD[_]] = { + def getPartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddValueOpt: Option[RDDValue] = _keyToPartitions.get(partitionKey) if (rddValueOpt.isDefined && _cachePolicy.isDefined) _cachePolicy.get.notifyGet(partitionKey) return rddValueOpt.map(_.rdd) @@ -72,10 +72,10 @@ class PartitionedMemoryTable( def putPartition( partitionKey: String, - newRDD: RDD[_], - isUpdate: Boolean = false): Option[RDD[_]] = { + newRDD: RDD[TablePartition], + isUpdate: Boolean = false): Option[RDD[TablePartition]] = { val rddValueOpt = _keyToPartitions.get(partitionKey) - var prevRDD: Option[RDD[_]] = rddValueOpt.map(_.rdd) + var prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) if (isUpdate && rddValueOpt.isDefined) { // This is an update of an old value, so update the RDDValue's 'rdd' entry. // Don't notify the '_cachePolicy'. Assumes that getPartition() has already been called to @@ -92,7 +92,7 @@ class PartitionedMemoryTable( return prevRDD } - def removePartition(partitionKey: String): Option[RDD[_]] = { + def removePartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) if (rddRemoved.isDefined && _cachePolicy.isDefined) { _cachePolicy.get.notifyRemove(partitionKey, rddRemoved.get) @@ -123,10 +123,10 @@ class PartitionedMemoryTable( _cachePolicy = Some(newPolicy) } - def cachePolicy: Option[CachePolicy[String, _]] = _cachePolicy + def cachePolicy: Option[CachePolicy[String, RDDValue]] = _cachePolicy /** Returns an immutable view of the String->RDD mapping to external callers */ - def keyToPartitions: collection.immutable.Map[String, RDD[_]] = { + def keyToPartitions: collection.immutable.Map[String, RDD[TablePartition]] = { return _keyToPartitions.mapValues(_.rdd).toMap } From 1f9c7a57f536412abb217913814ce4569f9e4e54 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 14 Oct 2013 08:30:53 -0700 Subject: [PATCH 120/331] Slightly revise some comments. --- .../execution/MemoryStoreSinkOperator.scala | 26 +++++++++--------- .../memstore2/MemoryMetadataManager.scala | 12 +++++---- .../shark/parse/SharkSemanticAnalyzer.scala | 27 ++++++++++--------- 3 files changed, 34 insertions(+), 31 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 97d9b5f2..ffeb372b 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -38,7 +38,7 @@ import shark.tachyon.TachyonTableWriter */ class MemoryStoreSinkOperator extends TerminalOperator { - // The initial capacity for ArrayLists used to represent columns in columnarcache. If -1, + // The initial capacity for ArrayLists used to construct the columnar storage. If -1, // the ColumnarSerde will obtain the partition size from a Configuration during execution // initialization (see ColumnarSerde#initialize()). @BeanProperty var partitionSize: Int = _ @@ -46,20 +46,20 @@ class MemoryStoreSinkOperator extends TerminalOperator { // If true, columnar storage will use compression. @BeanProperty var shouldCompress: Boolean = _ - // Storage level to use for the RDD created andmaterialized by this sink operator. + // Storage level to use for the RDD created and materialized by this sink operator. @BeanProperty var storageLevel: StorageLevel = _ - // For CTAS, this is the name of the table that will be created. For INSERTS, this is the name of + // For CTAS, this is the name of the table that is created. For INSERTS, this is the name of // the table that is modified. @BeanProperty var tableName: String = _ - // Used only for commands on Hive partitions. This partition key is a set of unique values for the - // the table's partitioning columns, and identifies the partition (represented by an RDD) that - // will be created or modified for an INSERT command. + // Used only for commands that target Hive partitions. The partition key is a set of unique values + // for the the table's partitioning columns and identifies the partition (represented by an RDD) + // that will be created or modified by the INSERT command being handled. @BeanProperty var hivePartitionKey: String = _ - // The memory storage used to store the output RDD - e.g., CacheType.HEAP refers to Spark's block - // manager. + // The memory storage used to store the output RDD - e.g., CacheType.HEAP refers to Spark's + // block manager. @transient var cacheMode: CacheType.CacheType = _ // Whether to compose a UnionRDD from the output RDD and a previous RDD. For example, for an @@ -125,7 +125,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned(tableName) // If true, a UnionRDD will be used to combine the RDD that contains the query output with the - // previous RDD, which is fetched using 'tableName' or, if the table is Hive-partitioned, the + // previous RDD, which is fetched using 'tableName' or - if the table is Hive-partitioned - a // ('tableName', 'hivePartitionKey') pair. var hasPreviousRDDForUnion = false @@ -167,12 +167,12 @@ class MemoryStoreSinkOperator extends TerminalOperator { } outputRDD = previousRDDOpt match { case Some(previousRDD) => { - // If the table or a Hive-partition that for the INSERT has already been created, - // take a union of the current data and the SELECT output. + // If the table or a Hive-partition for the INSERT has already been created, take a + // union of the current data and the SELECT output. hasPreviousRDDForUnion = true queryOutputRDD.union(previousRDD.asInstanceOf[RDD[TablePartition]]) } - // The 'previousRDDOpt' is None if this is an INSERT into a new Hive-partition. + // This is an INSERT into a new Hive-partition. case None => queryOutputRDD } } @@ -196,7 +196,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { } } else { outputRDD.setName(tableName) - // Create a new MemoryTable entry if one didn't exist (i.e., this operator is for a CTAS). + // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(tableName).getOrElse( SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode, storageLevel)) memoryTable.tableRDD = outputRDD diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 082a5167..2a9ff05b 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -132,13 +132,12 @@ class MemoryMetadataManager { /** * Used to drop a table from the Spark in-memory cache and/or disk. All metadata tracked by Shark - * tracked by Shark (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) is deleted - * as well. + * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) is deleted as well. * * @param key Name of the table to drop. * @return Option::isEmpty() is true of there is no MemoryTable (and RDD) corresponding to 'key' - * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will - * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. + * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will + * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ def removeTable(tableName: String): Option[RDD[_]] = { val lowerCaseTableName = tableName.toLowerCase @@ -176,7 +175,10 @@ object MemoryMetadataManager { return unpersistedRDD } - /** Return a string representation of the partition key, 'col1=value1/col2=value2/...' */ + /** + * Return a representation of the partition key in the string format: + * 'col1=value1/col2=value2/...' + */ def makeHivePartitionKeyStr( partitionColumns: Seq[String], partitionColumnToValue: JavaMap[String, String]): String = { diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index b94f2bef..0bf0a8f0 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -94,7 +94,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // HiveParer.HIVE_QUERY. child = queryStmtASTNode // Hive's super.analyzeInternal() might generate MapReduce tasks. Avoid executing those - // tasks by reset()-ing some Hive SemanticAnalyzer state after doPhase1(). + // tasks by reset()-ing some Hive SemanticAnalyzer state after doPhase1() is called below. shouldReset = true } case None => { @@ -107,7 +107,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } // Invariant: At this point, the command will execute a query (i.e., its AST contains a - // HiveParser.TOK_QUERY node). + // HiveParser.TOK_QUERY node). // Continue analyzing from the child ASTNode. if (!doPhase1(child, qb, initPhase1Ctx())) { @@ -364,7 +364,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } } - // Invariant: At this point, the command is either a CTAS or a CREATE TABLE. var ddlTasks: Seq[DDLTask] = Nil val createTableDesc = if (isRegularCreateTable) { @@ -372,17 +371,17 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // SemanticAnalyzer#analyzeCreateTable() does't set the CreateTableDesc in its QB. ddlTasks = rootTasks.filter(_.isInstanceOf[DDLTask]).asInstanceOf[Seq[DDLTask]] if (ddlTasks.isEmpty) null else ddlTasks.head.getWork.getCreateTblDesc - } - else { + } else { getParseContext.getQB.getTableDesc } - // Can be NULL if there is an IF NOTE EXISTS condition and the table already exists. + // 'createTableDesc' is NULL if there is an IF NOT EXISTS condition and the target table + // already exists. if (createTableDesc != null) { val tableName = createTableDesc.getTableName val checkTableName = SharkConfVars.getBoolVar(conf, SharkConfVars.CHECK_TABLENAME_FLAG) - // The CreateTableDesc's table properties are Java Maps, but the TableDesc's table properties, - // which are used during execution, are Java Properties. + // Note that the CreateTableDesc's table properties are Java Maps, but the TableDesc's table + // properties, which are used during execution, are Java Properties. val createTableProperties: JavaMap[String, String] = createTableDesc.getTblProps() // There are two cases that will enable caching: @@ -405,13 +404,15 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with createTableDesc.setSerName(classOf[ColumnarSerDe].getName) } - // For CTAS, the SparkTask's MemoryStoreSinkOperator will create the table and the Hive - // DDLTask will be a dependent of the SparkTask. SparkTasks are created in genMapRedTasks(). + // For CTAS ('isRegularCreateTable' is false), the MemoryStoreSinkOperator creates a new + // table metadata entry in the MemoryMetadataManager. The SparkTask that encloses the + // MemoryStoreSinkOperator will have a child Hive DDLTask, which creates a new table metadata + // entry in the Hive metastore. See genMapRedTasks() for SparkTask creation. if (isRegularCreateTable && shouldCache) { // In Hive, a CREATE TABLE command is handled by a DDLTask, created by - // SemanticAnalyzer#analyzeCreateTable(). The DDL tasks' execution succeeds only if the - // CREATE TABLE is valid. So, hook a SharkDDLTask as a dependent of the Hive DDLTask so that - // Shark metadata is updated only if the Hive task execution is successful. + // SemanticAnalyzer#analyzeCreateTable(), in 'rootTasks'. The DDL tasks' execution succeeds + // only if the CREATE TABLE is valid. So, hook a SharkDDLTask as a child of the Hive DDLTask + // so that Shark metadata is updated only if the Hive task execution is successful. val hiveDDLTask = ddlTasks.head; val sharkDDLWork = new SharkDDLWork(createTableDesc) sharkDDLWork.cacheMode = cacheMode From 2dd0c96725db9095f795c2bf3dffbb9f16396a66 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 14 Oct 2013 19:01:55 -0700 Subject: [PATCH 121/331] Add a NULL check in ReduceKeyReduceSide#equals(). --- src/main/scala/shark/execution/ReduceKey.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark/execution/ReduceKey.scala b/src/main/scala/shark/execution/ReduceKey.scala index d91df3bc..e2436b19 100755 --- a/src/main/scala/shark/execution/ReduceKey.scala +++ b/src/main/scala/shark/execution/ReduceKey.scala @@ -79,7 +79,8 @@ class ReduceKeyMapSide(var bytesWritable: BytesWritable) extends ReduceKey if (length != other.length) { false } else { - WritableComparator.compareBytes(byteArray, 0, length, other.byteArray, 0, other.length) == 0 + WritableComparator.compareBytes( + byteArray, 0, length, other.byteArray, 0, other.length) == 0 } } case _ => false @@ -116,10 +117,12 @@ class ReduceKeyReduceSide(private val _byteArray: Array[Byte]) extends ReduceKey override def length: Int = byteArray.length override def equals(other: Any): Boolean = { - // We expect this is only used in a hash table comparing to the same types. - // So we force a type cast. - val that = other.asInstanceOf[ReduceKeyReduceSide] - (this.byteArray.length == that.byteArray.length && this.compareTo(that) == 0) + other match { + case that: ReduceKeyReduceSide => { + (this.byteArray.length == that.byteArray.length) && (this.compareTo(that) == 0) + } + case _ => false + } } override def compareTo(that: ReduceKey): Int = { From bae19222b3b221946ff870e0cee4dba0371dea04 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 14 Oct 2013 17:54:33 -0700 Subject: [PATCH 122/331] Revert to using RDD#collect() for map joins, Since Spark can send Task results through the block manager when the Akka frame size is exceeded - Reverts Shark PR #98 - Relevant commit on the Spark side: apache/incubator-spark@c75eb14 --- .../shark/execution/MapJoinOperator.scala | 24 ++++--------------- 1 file changed, 5 insertions(+), 19 deletions(-) diff --git a/src/main/scala/shark/execution/MapJoinOperator.scala b/src/main/scala/shark/execution/MapJoinOperator.scala index 400415b5..ea232b6e 100755 --- a/src/main/scala/shark/execution/MapJoinOperator.scala +++ b/src/main/scala/shark/execution/MapJoinOperator.scala @@ -28,9 +28,7 @@ import org.apache.hadoop.hive.ql.exec.{MapJoinOperator => HiveMapJoinOperator} import org.apache.hadoop.hive.ql.plan.MapJoinDesc import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import org.apache.spark.SparkEnv import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel import shark.SharkEnv import shark.execution.serialization.{OperatorSerializationWrapper, SerializableWritable} @@ -114,6 +112,7 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato // following mapParititons will fail because it tries to include the // outer closure, which references "this". val op = op1 + // An RDD of (Join key, Corresponding rows) tuples. val rddForHash: RDD[(Seq[AnyRef], Seq[Array[AnyRef]])] = rdd.mapPartitions { partition => op.initializeOnSlave() @@ -125,28 +124,15 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato // Collect the RDD and build a hash table. val startCollect = System.currentTimeMillis() - val storageLevel = rddForHash.getStorageLevel - if(storageLevel == StorageLevel.NONE) - rddForHash.persist(StorageLevel.MEMORY_AND_DISK) - rddForHash.foreach(_ => Unit) - val wrappedRows = rddForHash.partitions.flatMap { part => - val blockId = "rdd_%s_%s".format(rddForHash.id, part.index) - val iter = SparkEnv.get.blockManager.get(blockId) - val partRows = new ArrayBuffer[(Seq[AnyRef], Seq[Array[AnyRef]])] - iter.foreach(_.foreach { row => - partRows += row.asInstanceOf[(Seq[AnyRef], Seq[Array[AnyRef]])] - }) - partRows - } - if(storageLevel == StorageLevel.NONE) - rddForHash.unpersist() - logDebug("wrappedRows size:" + wrappedRows.size) + val collectedRows: Array[(Seq[AnyRef], Seq[Array[AnyRef]])] = rddForHash.collect() + + logDebug("collectedRows size:" + collectedRows.size) val collectTime = System.currentTimeMillis() - startCollect logInfo("HashTable collect took " + collectTime + " ms") // Build the hash table. - val hash = wrappedRows.groupBy(x => x._1) + val hash = collectedRows.groupBy(x => x._1) .mapValues(v => v.flatMap(t => t._2)) val map = new JHashMap[Seq[AnyRef], Array[Array[AnyRef]]]() From 67626ae3eb6a23efc504edf5aedc417197f072cf Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 15 Oct 2013 18:15:23 -0700 Subject: [PATCH 123/331] Get Shark to compile with latest Spark master. --- src/main/scala/shark/execution/CoGroupedRDD.scala | 2 +- src/main/scala/shark/execution/FileSinkOperator.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/execution/CoGroupedRDD.scala b/src/main/scala/shark/execution/CoGroupedRDD.scala index 4b5da641..8b212da4 100644 --- a/src/main/scala/shark/execution/CoGroupedRDD.scala +++ b/src/main/scala/shark/execution/CoGroupedRDD.scala @@ -122,7 +122,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) // Read map outputs of shuffle def mergePair(pair: (K, Any)) { getSeq(pair._1)(depNum) += pair._2 } val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[(K, Seq[Any])](shuffleId, split.index, context.taskMetrics, serializer) + fetcher.fetch[(K, Seq[Any])](shuffleId, split.index, context, serializer) .foreach(mergePair) } } diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index 641c827a..118a18a0 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -73,6 +73,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { iter.foreach { row => numRows += 1 + // Process and writes each row to a temp file. localHiveOp.processOp(row, 0) } @@ -112,7 +113,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { } } - localHiveOp.closeOp(false) + localHiveOp.closeOp(false /* abort */) Iterator(numRows) } @@ -181,7 +182,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { logDebug("Total number of rows written: " + rows.sum) } - hiveOp.jobClose(localHconf, true, new JobCloseFeedBack) + hiveOp.jobClose(localHconf, true /* success */, new JobCloseFeedBack) rdd } } From 493e7bf3e0ebf07c7c310323a420b6490fb2c73d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 15 Oct 2013 17:39:07 -0700 Subject: [PATCH 124/331] Remove Kryo references in SharkEnv. The ShuffleSerializer is explicitly specified as the object serializer for shuffle ops. --- src/main/scala/shark/SharkEnv.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 3e5f9d00..0d3148bb 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -85,9 +85,6 @@ object SharkEnv extends LogHelper { logDebug("Initializing SharkEnv") - System.setProperty("spark.serializer", classOf[SparkKryoSerializer].getName) - System.setProperty("spark.kryo.registrator", classOf[KryoRegistrator].getName) - val executorEnvVars = new HashMap[String, String] executorEnvVars.put("SCALA_HOME", getEnv("SCALA_HOME")) executorEnvVars.put("SPARK_MEM", getEnv("SPARK_MEM")) From 83482b2a1f8f47ca5597088c8a63f5ffdab6cb50 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Wed, 16 Oct 2013 18:59:45 -0700 Subject: [PATCH 125/331] Code cleanup. This code is not used and the intended optimization is covered else where. This code/tests cause issues with hadoop 0.23.9. Cleaning up. --- .../serialization/HiveConfSerializer.scala | 54 ------------------- .../serialization/SerializationSuite.scala | 8 --- 2 files changed, 62 deletions(-) delete mode 100644 src/main/scala/shark/execution/serialization/HiveConfSerializer.scala diff --git a/src/main/scala/shark/execution/serialization/HiveConfSerializer.scala b/src/main/scala/shark/execution/serialization/HiveConfSerializer.scala deleted file mode 100644 index db612e4c..00000000 --- a/src/main/scala/shark/execution/serialization/HiveConfSerializer.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.execution.serialization - -import java.io.ByteArrayInputStream -import java.io.ByteArrayOutputStream -import java.io.DataInputStream -import java.io.DataOutputStream - -import com.ning.compress.lzf.LZFEncoder -import com.ning.compress.lzf.LZFDecoder - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.io.Text - - -object HiveConfSerializer { - - def serialize(hConf: HiveConf): Array[Byte] = { - val os = new ByteArrayOutputStream - val dos = new DataOutputStream(os) - val auxJars = hConf.getAuxJars() - Text.writeString(dos, if(auxJars == null) "" else auxJars) - hConf.write(dos) - LZFEncoder.encode(os.toByteArray()) - } - - def deserialize(b: Array[Byte]): HiveConf = { - val is = new ByteArrayInputStream(LZFDecoder.decode(b)) - val dis = new DataInputStream(is) - val auxJars = Text.readString(dis) - val conf = new HiveConf - conf.readFields(dis) - if(auxJars.equals("").unary_!) - conf.setAuxJars(auxJars) - conf - } -} diff --git a/src/test/scala/shark/execution/serialization/SerializationSuite.scala b/src/test/scala/shark/execution/serialization/SerializationSuite.scala index f7f887e1..90383a2a 100755 --- a/src/test/scala/shark/execution/serialization/SerializationSuite.scala +++ b/src/test/scala/shark/execution/serialization/SerializationSuite.scala @@ -59,14 +59,6 @@ class SerializationSuite extends FunSuite { assert(desered.head.getTypeName() === oi.getTypeName()) } - test("HiveConf serialization test") { - val hiveConf = new HiveConf - val bytes = HiveConfSerializer.serialize(hiveConf) - val deseredConf = HiveConfSerializer.deserialize(bytes) - - assertHiveConfEquals(hiveConf, deseredConf) - } - test("Java serializing operators") { import shark.execution.{FileSinkOperator => SharkFileSinkOperator} From 49d4bb775afd0e50b2a6b29f448afc16a6d38c18 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 14 Oct 2013 23:17:38 -0700 Subject: [PATCH 126/331] Account for new Spark HadoopRDD API. Also, for non-partitioned tables stored in Hadoop, moves row deserialization from processPartition() to preprocessRDD(). This makes it more consistent with partitioned table reads/processing, and allows us to annotate HiveConf as transient, which should reduce Spark task sizes. --- .../shark/execution/TableScanOperator.scala | 227 +++++++++++++----- 1 file changed, 161 insertions(+), 66 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 6944b2cb..ae4d8a7e 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -18,7 +18,10 @@ package shark.execution import java.util.{ArrayList, Arrays} + +import scala.collection.JavaConversions._ import scala.reflect.BeanProperty + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.hadoop.hive.conf.HiveConf @@ -27,17 +30,18 @@ import org.apache.hadoop.hive.ql.exec.{TableScanOperator => HiveTableScanOperato import org.apache.hadoop.hive.ql.exec.{MapSplitPruning, Utilities} import org.apache.hadoop.hive.ql.io.HiveInputFormat import org.apache.hadoop.hive.ql.metadata.{Partition, Table} -import org.apache.hadoop.hive.ql.plan.{PlanUtils, PartitionDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.{PlanUtils, PartitionDesc, TableDesc, TableScanDesc} +import org.apache.hadoop.hive.serde.Constants import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.io.Writable import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.{PartitionPruningRDD, RDD, UnionRDD} +import org.apache.spark.rdd.{HadoopRDD, PartitionPruningRDD, RDD, UnionRDD} import org.apache.spark.SerializableWritable -import shark.{SharkConfVars, SharkEnv, Utils} +import shark.{LogHelper, SharkConfVars, SharkEnv, Utils} import shark.api.QueryExecutionException import shark.execution.optimization.ColumnPruner import shark.execution.serialization.{XmlSerializer, JavaSerializer} @@ -56,6 +60,9 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // table isn't Hive-partitioned. Set in SparkTask::initializeTableScanTableDesc(). @transient var parts: Array[Object] = _ + // For convenience, a local copy of the HiveConf for this task. + @transient var localHConf: HiveConf = _ + // PartitionDescs are used during planning in Hive. This reference to a single PartitionDesc // is used to initialize partition ObjectInspectors. // If the table is not Hive-partitioned, then 'firstConfPartDesc' won't be used. The value is not @@ -65,7 +72,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO @BeanProperty var firstConfPartDesc: PartitionDesc = _ @BeanProperty var tableDesc: TableDesc = _ - @BeanProperty var localHconf: HiveConf = _ + /** * Initialize the hive TableScanOperator. This initialization propagates @@ -108,7 +115,9 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } override def initializeOnMaster() { - localHconf = super.hconf + // Create a local copy of the HiveConf that will be assigned job properties and, for disk reads, + // broadcasted to slaves. + localHConf = new HiveConf(super.hconf) } override def execute(): RDD[_] = { @@ -166,13 +175,13 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO SharkEnv.tachyonUtil.pushDownColumnPruning(rdd, columnsUsed) val prunedRdd: RDD[_] = - if (SharkConfVars.getBoolVar(localHconf, SharkConfVars.MAP_PRUNING) && + if (SharkConfVars.getBoolVar(localHConf, SharkConfVars.MAP_PRUNING) && childOperators(0).isInstanceOf[FilterOperator] && indexToStats.size == rdd.partitions.size) { val startTime = System.currentTimeMillis val printPruneDebug = SharkConfVars.getBoolVar( - localHconf, SharkConfVars.MAP_PRUNING_PRINT_DEBUG) + localHConf, SharkConfVars.MAP_PRUNING_PRINT_DEBUG) // Must initialize the condition evaluator in FilterOperator to get the // udfs and object inspectors set. @@ -214,69 +223,62 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } /** - * Create a RDD representing the table (with or without partitions). + * Create a RDD for a table. */ override def preprocessRdd(rdd: RDD[_]): RDD[_] = { - // copyTableJobPropertiesToConf and pushFilters both take in JobConfs. - val localJobConf = new JobConf(localHconf) - if (tableDesc != null) { - Utilities.copyTableJobPropertiesToConf(tableDesc, localJobConf) - } - val bufferSize = System.getProperty("spark.buffer.size", "65536") - localJobConf.set("io.file.buffer.size", bufferSize) - - new HiveInputFormat() { - def doPushFilters() { - pushFilters(localJobConf, hiveOp) - } - }.doPushFilters() - - // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate - // from Spark's master all the way to Spark's slaves. - var s3varsSet = false - val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", - "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => - if (localJobConf.get(variableName) != null) { - s3varsSet = true - localJobConf.set(variableName, localJobConf.get(variableName)) - } - } - - // If none of the s3 credentials are set in Hive conf, try use the environmental - // variables for credentials. - if (!s3varsSet) { - Utils.setAwsCredentials(localJobConf) - } - // Choose the minimum number of splits. If mapred.map.tasks is set, use that unless // it is smaller than what Spark suggests. val minSplitsPerRDD = math.max( - localJobConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) + localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) - val broadcastedJobConf = SharkEnv.sc.broadcast(new SerializableWritable(localJobConf)) + TableScanOperator.prepareHiveConf(localHConf, hiveOp) + val broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(localHConf)) if (table.isPartitioned) { logDebug("Making %d Hive partitions".format(parts.size)) - makeHivePartitionRDDs(broadcastedJobConf, minSplitsPerRDD) + makeHivePartitionRDDs(broadcastedHiveConf, minSplitsPerRDD) } else { - val tablePath = table.getPath.toString - val ifc = table.getInputFormatClass - .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - logDebug("Table input: %s".format(tablePath)) - createHadoopFileRdd(tablePath, ifc, broadcastedJobConf, minSplitsPerRDD) + makeTableRdd(broadcastedHiveConf, minSplitsPerRDD) } } - override def processPartition(index: Int, iter: Iterator[_]): Iterator[_] = { - val deserializer = tableDesc.getDeserializerClass().newInstance() - deserializer.initialize(localHconf, tableDesc.getProperties) - iter.map { value => - value match { - case rowWithPart: Array[Object] => rowWithPart - case v: Writable => deserializer.deserialize(v) - case _ => throw new RuntimeException("Failed to match " + value.toString) + /** + * Forward all rows. TableScanOperator doesn't need to do any more processing of values read and + * preprocessed (i.e., deserialized) from disk. + * + * For Hive-partitioned tables, the iterator returns two-element arrays with the elements as + * (deserialized row, column partition value). For non-partitioned tables, the iterator returns + * deserialized row Objects. + */ + override def processPartition(index: Int, iter: Iterator[_]): Iterator[_] = iter + + /** + * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed + * RDD that contains deserialized rows. + */ + private def makeTableRdd( + broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], + minSplits: Int): RDD[_] = { + val tablePath = table.getPath.toString + val ifc = table.getInputFormatClass + .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] + logDebug("Table input: %s".format(tablePath)) + + val hadoopRDD = createHadoopRdd(tablePath, ifc, broadcastedHiveConf, minSplits) + val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => + val hconf = broadcastedHiveConf.value.value + val deserializer = tableDesc.getDeserializerClass().newInstance() + deserializer.initialize(hconf, tableDesc.getProperties) + + // Deserialize each Writable to get the row value. + iter.map { value => + value match { + case v: Writable => deserializer.deserialize(v) + case _ => throw new RuntimeException("Failed to match " + value.toString) + } } } + return deserializedHadoopRDD } /** @@ -285,7 +287,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO * 'PARTITION BY'. */ private def makeHivePartitionRDDs[T]( - broadcastedJobConf: Broadcast[SerializableWritable[JobConf]], + broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], minSplitsPerRDD: Int ): RDD[_] = { val partitions = parts @@ -299,12 +301,11 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO val ifc = partition.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - val hivePartitionRDD = createHadoopFileRdd( - tablePath, ifc, broadcastedJobConf, minSplitsPerRDD) + val hivePartitionRDD = createHadoopRdd( + tablePath, ifc, broadcastedHiveConf, minSplitsPerRDD) - val serializedHconf = XmlSerializer.serialize(localHconf, localHconf) val hivePartitionRDDWithColValues = hivePartitionRDD.mapPartitions { iter => - val hconf = XmlSerializer.deserialize(serializedHconf).asInstanceOf[HiveConf] + val hconf = broadcastedHiveConf.value.value val deserializer = partDesc.getDeserializerClass().newInstance() deserializer.initialize(hconf, partDesc.getProperties()) @@ -345,16 +346,35 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } } - private def createHadoopFileRdd( + /** + * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be + * applied locally on each slave. + */ + private def createHadoopRdd( path: String, inputFormatClass: Class[InputFormat[Writable, Writable]], - broadcastedJobConf: Broadcast[SerializableWritable[JobConf]], + broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], minSplits: Int) - : RDD[Writable] = { + : RDD[Writable] = { + /* + * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to + * instantiate a HadoopRDD. + */ + def initializeLocalJobConfFunc(path: String)(jobConf: JobConf) { + FileInputFormat.setInputPaths(jobConf, path) + if (tableDesc != null) { + Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) + } + val bufferSize = System.getProperty("spark.buffer.size", "65536") + jobConf.set("io.file.buffer.size", bufferSize) + } + + val initializeJobConfFunc = initializeLocalJobConfFunc(path) _ - val rdd = SharkEnv.sc.hadoopFile( - path, - broadcastedJobConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], + val rdd = new HadoopRDD( + SharkEnv.sc, + broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], + Some(initializeJobConfFunc), inputFormatClass, classOf[Writable], classOf[Writable], @@ -364,3 +384,78 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO rdd.map(_._2) } } + + +object TableScanOperator extends LogHelper { + + /** + * Add miscellaneous properties to the HiveConf to be used for creating a HadoopRDD. These + * properties are impractical to add during local JobConf creation in HadoopRDD - for example, + * filter expressions would require a serialized HiveTableScanOperator. + */ + private def prepareHiveConf(hiveConf: HiveConf, hiveTableScanOp: HiveTableScanOperator) { + // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate + // from Spark's master all the way to Spark's slaves. + var s3varsSet = false + val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", + "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => + if (hiveConf.get(variableName) != null) { + s3varsSet = true + } + } + + // If none of the s3 credentials are set in Hive conf, try use the environmental + // variables for credentials. + if (!s3varsSet) { + Utils.setAwsCredentials(hiveConf) + } + + TableScanOperator.addFilterExprToConf(hiveConf, hiveTableScanOp) + } + + /** + * Add filter expressions and column metadata to the HiveConf This is meant to be called on the + * master, so that we can avoid serializing the HiveTableScanOperator. + */ + private def addFilterExprToConf(hiveConf: HiveConf, hiveTableScanOp: HiveTableScanOperator) { + val tableScanDesc = hiveTableScanOp.getConf() + if (tableScanDesc == null) return + + val rowSchema = hiveTableScanOp.getSchema + if (rowSchema != null) { + // Add column names to the HiveConf. + var columnNames = new StringBuilder() + for (columnInfo <- rowSchema.getSignature()) { + if (columnNames.length() > 0) { + columnNames.append(",") + } + columnNames.append(columnInfo.getInternalName()) + } + val columnNamesString = columnNames.toString(); + hiveConf.set(Constants.LIST_COLUMNS, columnNamesString); + + // Add column types to the HiveConf. + var columnTypes = new StringBuilder() + for (columnInfo <- rowSchema.getSignature()) { + if (columnTypes.length() > 0) { + columnTypes.append(",") + } + columnTypes.append(columnInfo.getType().getTypeName()) + } + val columnTypesString = columnTypes.toString + hiveConf.set(Constants.LIST_COLUMN_TYPES, columnTypesString) + } + + // Push down predicate filters. + val filterExprNode = tableScanDesc.getFilterExpr() + if (filterExprNode == null) return + + val filterText = filterExprNode.getExprString() + hiveConf.set(TableScanDesc.FILTER_TEXT_CONF_STR, filterText) + logDebug("Filter text: " + filterText) + + val filterExprNodeSerialized = Utilities.serializeExpression(filterExprNode) + hiveConf.set(TableScanDesc.FILTER_EXPR_CONF_STR, filterExprNodeSerialized) + logDebug("Filter expression: " + filterExprNodeSerialized) + } +} From 699eb5bfceaa2ec4702c1e7e324fa1886b8e0324 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 15 Oct 2013 17:39:07 -0700 Subject: [PATCH 127/331] Remove Kryo references in SharkEnv. The ShuffleSerializer is explicitly specified as the object serializer for shuffle ops. --- src/main/scala/shark/SharkEnv.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 3e5f9d00..0d3148bb 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -85,9 +85,6 @@ object SharkEnv extends LogHelper { logDebug("Initializing SharkEnv") - System.setProperty("spark.serializer", classOf[SparkKryoSerializer].getName) - System.setProperty("spark.kryo.registrator", classOf[KryoRegistrator].getName) - val executorEnvVars = new HashMap[String, String] executorEnvVars.put("SCALA_HOME", getEnv("SCALA_HOME")) executorEnvVars.put("SPARK_MEM", getEnv("SPARK_MEM")) From 97837281cf7011ff9b6baf865a3becbe8c4d599f Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 15 Oct 2013 18:15:23 -0700 Subject: [PATCH 128/331] Get Shark to compile with latest Spark master. --- src/main/scala/shark/execution/CoGroupedRDD.scala | 2 +- src/main/scala/shark/execution/FileSinkOperator.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/execution/CoGroupedRDD.scala b/src/main/scala/shark/execution/CoGroupedRDD.scala index 4b5da641..8b212da4 100644 --- a/src/main/scala/shark/execution/CoGroupedRDD.scala +++ b/src/main/scala/shark/execution/CoGroupedRDD.scala @@ -122,7 +122,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) // Read map outputs of shuffle def mergePair(pair: (K, Any)) { getSeq(pair._1)(depNum) += pair._2 } val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[(K, Seq[Any])](shuffleId, split.index, context.taskMetrics, serializer) + fetcher.fetch[(K, Seq[Any])](shuffleId, split.index, context, serializer) .foreach(mergePair) } } diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index 641c827a..118a18a0 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -73,6 +73,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { iter.foreach { row => numRows += 1 + // Process and writes each row to a temp file. localHiveOp.processOp(row, 0) } @@ -112,7 +113,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { } } - localHiveOp.closeOp(false) + localHiveOp.closeOp(false /* abort */) Iterator(numRows) } @@ -181,7 +182,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { logDebug("Total number of rows written: " + rows.sum) } - hiveOp.jobClose(localHconf, true, new JobCloseFeedBack) + hiveOp.jobClose(localHconf, true /* success */, new JobCloseFeedBack) rdd } } From 9eafbf6a112c6147f9a130c9a03b0bde80970dc4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 17 Oct 2013 15:03:54 -0700 Subject: [PATCH 129/331] Added a release cleanup script. --- bin/dev/release_cleanup.sh | 40 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100755 bin/dev/release_cleanup.sh diff --git a/bin/dev/release_cleanup.sh b/bin/dev/release_cleanup.sh new file mode 100755 index 00000000..abb07b6a --- /dev/null +++ b/bin/dev/release_cleanup.sh @@ -0,0 +1,40 @@ +#!/bin/sh + +# Copyright (C) 2012 The Regents of The University California. +# All rights reserved. +# +# Licensed 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. + +DEVDIR="`dirname $0`" +BINDIR="`dirname $DEVDIR`" +FWDIR="`dirname $BINDIR`" + +rm -rf $FWDIR/run-tests-from-scratch-workspace +rm -rf $FWDIR/test_warehouses + +rm -rf $FWDIR/conf/shark-env.sh + +rm -rf $FWDIR/metastore_db +rm -rf $FWDIR/derby.log + +rm -rf $FWDIR/project/target $FWDIR/project/project/target + +rm -rf $FWDIR/target/resolution-cache +rm -rf $FWDIR/target/streams +rm -rf $FWDIR/target/scala-*/cache +rm -rf $FWDIR/target/scala-*/classes +rm -rf $FWDIR/target/scala-*/test-classes + +find $FWDIR -name ".DS_Store" -exec rm {} \; +find $FWDIR -name ".history" -exec rm {} \; + From fa499012cf3560a9b2464a5b99c266e86e8c6e68 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 17 Oct 2013 15:44:50 -0700 Subject: [PATCH 130/331] Remove unused HIVE_WAREHOUSE var in run-tests-from-scratch This was also creating an empty, unused ../hive-warehouse directory. --- bin/dev/run-tests-from-scratch | 6 ------ 1 file changed, 6 deletions(-) diff --git a/bin/dev/run-tests-from-scratch b/bin/dev/run-tests-from-scratch index a60256e6..d3a7192b 100755 --- a/bin/dev/run-tests-from-scratch +++ b/bin/dev/run-tests-from-scratch @@ -279,12 +279,6 @@ if $SKIP_HIVE ; then exit -1 fi else - # Setup the Hive warehouse directory. - HIVE_WAREHOUSE=./hive-warehouse - rm -rf $HIVE_WAREHOUSE - mkdir -p $HIVE_WAREHOUSE - chmod 0777 $HIVE_WAREHOUSE - rm -rf hive git clone $HIVE_GIT_URL pushd hive From a74d23e3982a47867e92f3557f70f8c7cc25736f Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 17 Oct 2013 15:49:12 -0700 Subject: [PATCH 131/331] In run-tests-from-scratch, no need to check for 'hive-warehouse' directory existence when skipping Hive build. --- bin/dev/run-tests-from-scratch | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/dev/run-tests-from-scratch b/bin/dev/run-tests-from-scratch index d3a7192b..a19a5898 100755 --- a/bin/dev/run-tests-from-scratch +++ b/bin/dev/run-tests-from-scratch @@ -274,8 +274,8 @@ export HADOOP_HOME="$WORKSPACE/hadoop-${SPARK_HADOOP_VERSION}" # Download and build Hive. ##################################################################### if $SKIP_HIVE ; then - if [ ! -e "hive" -o ! -e "hive-warehouse" ] ; then - echo "hive and hive-warehouse dirs must exist when skipping Hive download and build stage." + if [ ! -e "hive" ] ; then + echo "hive dir must exist when skipping Hive download and build stage." exit -1 fi else From 43360cc129906528aa00b8fc4accb63ec3a5e68d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 17 Oct 2013 17:36:42 -0700 Subject: [PATCH 132/331] Don't use Operator#execute() in TableScanOperator. Plus a few other minor style changes. --- .../shark/execution/TableScanOperator.scala | 47 +++++++++---------- 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index ae4d8a7e..b822c64b 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -157,10 +157,10 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO logInfo("Loading table " + tableKey + " stats from Tachyon.") SharkEnv.memoryMetadataManager.putStats(tableKey, indexToStats) } - createPrunedRdd(tableKey, SharkEnv.tachyonUtil.createRDD(tableKey)) + return createPrunedRdd(tableKey, SharkEnv.tachyonUtil.createRDD(tableKey)) } else { // Table is a Hive table on HDFS (or other Hadoop storage). - super.execute() + return makeRDDFromHadoop() } } @@ -225,7 +225,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO /** * Create a RDD for a table. */ - override def preprocessRdd(rdd: RDD[_]): RDD[_] = { + def makeRDDFromHadoop(): RDD[_] = { // Choose the minimum number of splits. If mapred.map.tasks is set, use that unless // it is smaller than what Spark suggests. val minSplitsPerRDD = math.max( @@ -236,27 +236,20 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO if (table.isPartitioned) { logDebug("Making %d Hive partitions".format(parts.size)) - makeHivePartitionRDDs(broadcastedHiveConf, minSplitsPerRDD) + // The returned RDD contains arrays of size two with the elements as + // (deserialized row, column partition value). + return makeHivePartitionRDDs(broadcastedHiveConf, minSplitsPerRDD) } else { - makeTableRdd(broadcastedHiveConf, minSplitsPerRDD) + // The returned RDD contains deserialized row Objects. + return makeTableRDD(broadcastedHiveConf, minSplitsPerRDD) } } - /** - * Forward all rows. TableScanOperator doesn't need to do any more processing of values read and - * preprocessed (i.e., deserialized) from disk. - * - * For Hive-partitioned tables, the iterator returns two-element arrays with the elements as - * (deserialized row, column partition value). For non-partitioned tables, the iterator returns - * deserialized row Objects. - */ - override def processPartition(index: Int, iter: Iterator[_]): Iterator[_] = iter - /** * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed * RDD that contains deserialized rows. */ - private def makeTableRdd( + private def makeTableRDD( broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], minSplits: Int): RDD[_] = { val tablePath = table.getPath.toString @@ -383,6 +376,10 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) } + + // All RDD processing is done in execute(). + override def processPartition(split: Int, iter: Iterator[_]): Iterator[_] = + throw new UnsupportedOperationException("TableScanOperator.processPartition()") } @@ -448,14 +445,14 @@ object TableScanOperator extends LogHelper { // Push down predicate filters. val filterExprNode = tableScanDesc.getFilterExpr() - if (filterExprNode == null) return - - val filterText = filterExprNode.getExprString() - hiveConf.set(TableScanDesc.FILTER_TEXT_CONF_STR, filterText) - logDebug("Filter text: " + filterText) - - val filterExprNodeSerialized = Utilities.serializeExpression(filterExprNode) - hiveConf.set(TableScanDesc.FILTER_EXPR_CONF_STR, filterExprNodeSerialized) - logDebug("Filter expression: " + filterExprNodeSerialized) + if (filterExprNode != null) { + val filterText = filterExprNode.getExprString() + hiveConf.set(TableScanDesc.FILTER_TEXT_CONF_STR, filterText) + logDebug("Filter text: " + filterText) + + val filterExprNodeSerialized = Utilities.serializeExpression(filterExprNode) + hiveConf.set(TableScanDesc.FILTER_EXPR_CONF_STR, filterExprNodeSerialized) + logDebug("Filter expression: " + filterExprNodeSerialized) + } } } From 1fce45302e2a82f80451db647937798508234334 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 17 Oct 2013 18:50:22 -0700 Subject: [PATCH 133/331] Enable query cancellation in local CLI. --- src/main/scala/shark/SharkCliDriver.scala | 51 +++++++++++++---------- 1 file changed, 30 insertions(+), 21 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 2dd66073..660a5bbd 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -25,33 +25,40 @@ import java.io.PrintStream import java.io.UnsupportedEncodingException import java.net.URLClassLoader import java.util.ArrayList -import jline.{History, ConsoleReader} + import scala.collection.JavaConversions._ +import jline.{History, ConsoleReader} + import org.apache.commons.lang.StringUtils import org.apache.commons.logging.LogFactory import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} -import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, Utilities} -import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.parse.ParseDriver +import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.io.IOUtils -import org.apache.spark.SparkContext - object SharkCliDriver { - var prompt = "shark" - var prompt2 = " " // when ';' is not yet seen. + private var prompt = "shark" + private var prompt2 = " " // when ';' is not yet seen. + + installSignalHandler() + + def installSignalHandler() { + HiveInterruptUtils.add(new HiveInterruptCallback { + override def interrupt() { + SharkEnv.sc.cancelAllJobs() + } + }) + } def main(args: Array[String]) { val hiveArgs = args.filterNot(_.equals("-loadRdds")) @@ -73,11 +80,11 @@ object SharkCliDriver { logInitDetailMessage = e.getMessage() } - var ss = new CliSessionState(new HiveConf(classOf[SessionState])) + val ss = new CliSessionState(new HiveConf(classOf[SessionState])) ss.in = System.in try { ss.out = new PrintStream(System.out, true, "UTF-8") - ss.info = new PrintStream(System.err, true, "UTF-8"); + ss.info = new PrintStream(System.err, true, "UTF-8") ss.err = new PrintStream(System.err, true, "UTF-8") } catch { case e: UnsupportedEncodingException => System.exit(3) @@ -134,7 +141,7 @@ object SharkCliDriver { Thread.currentThread().setContextClassLoader(loader) } - var cli = new SharkCliDriver(loadRdds) + val cli = new SharkCliDriver(loadRdds) cli.setHiveVariables(oproc.getHiveVariables()) // Execute -i init files (always in silent mode) @@ -154,7 +161,7 @@ object SharkCliDriver { System.exit(3) } - var reader = new ConsoleReader() + val reader = new ConsoleReader() reader.setBellEnabled(false) // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) reader.addCompletor(CliDriver.getCommandCompletor()) @@ -163,7 +170,7 @@ object SharkCliDriver { val HISTORYFILE = ".hivehistory" val historyDirectory = System.getProperty("user.home") try { - if ((new File(historyDirectory)).exists()) { + if (new File(historyDirectory).exists()) { val historyFile = historyDirectory + File.separator + HISTORYFILE reader.setHistory(new History(new File(historyFile))) } else { @@ -189,7 +196,7 @@ object SharkCliDriver { var ret = 0 var prefix = "" - var curDB = getFormattedDbMethod.invoke(null, conf, ss).asInstanceOf[String] + val curDB = getFormattedDbMethod.invoke(null, conf, ss).asInstanceOf[String] var curPrompt = SharkCliDriver.prompt + curDB var dbSpaces = spacesForStringMethod.invoke(null, curDB).asInstanceOf[String] @@ -200,7 +207,7 @@ object SharkCliDriver { } if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { line = prefix + line - ret = cli.processLine(line) + ret = cli.processLine(line, true) prefix = "" val sharkMode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "shark" curPrompt = if (sharkMode) SharkCliDriver.prompt else CliDriver.prompt @@ -216,7 +223,7 @@ object SharkCliDriver { ss.close() System.exit(ret) - } + } // end of main } @@ -230,7 +237,7 @@ class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper private val conf: Configuration = if (ss != null) ss.getConf() else new Configuration() - SharkConfVars.initializeWithDefaults(conf); + SharkConfVars.initializeWithDefaults(conf) // Force initializing SharkEnv. This is put here but not object SharkCliDriver // because the Hive unit tests do not go through the main() code path. @@ -238,7 +245,9 @@ class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper SharkEnv.init() } - if(loadRdds) CachedTableRecovery.loadAsRdds(processCmd(_)) + if (loadRdds) { + CachedTableRecovery.loadAsRdds(processCmd(_)) + } def this() = this(false) @@ -309,7 +318,7 @@ class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper try { while (!out.checkError() && qp.getResults(res)) { - res.foreach(out.println(_)) + res.foreach(line => out.println(line)) res.clear() } } catch { From c6ceca3b5994d9f11eef0fbbd78debb45ceb7420 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 17 Oct 2013 18:52:53 -0700 Subject: [PATCH 134/331] Added a commnet to explain how the signal handler works. --- src/main/scala/shark/SharkCliDriver.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 660a5bbd..5c03e8c5 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -52,6 +52,10 @@ object SharkCliDriver { installSignalHandler() + /** + * Install a interrupt callback to cancel all Spark jobs. In Hive CliDriver's processLine, it + * installs a signal handler for ctrl + c to invoke callbacks defined in HiveInterruptUtils. + */ def installSignalHandler() { HiveInterruptUtils.add(new HiveInterruptCallback { override def interrupt() { From 72def864113ebaaee8e09fb1ac6d254eba38e77e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 17 Oct 2013 21:50:34 -0700 Subject: [PATCH 135/331] Updated query cancellation comment according to Harvey's feedback. --- src/main/scala/shark/SharkCliDriver.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 5c03e8c5..06e2b3d0 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -53,8 +53,9 @@ object SharkCliDriver { installSignalHandler() /** - * Install a interrupt callback to cancel all Spark jobs. In Hive CliDriver's processLine, it - * installs a signal handler for ctrl + c to invoke callbacks defined in HiveInterruptUtils. + * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), + * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while + * a command is being processed by the current thread. */ def installSignalHandler() { HiveInterruptUtils.add(new HiveInterruptCallback { From b0fcbe8929a5f6b931a17c2bd170b04467894d1e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 18 Oct 2013 22:29:52 -0700 Subject: [PATCH 136/331] Minor style cleanup. --- src/main/scala/shark/SharkEnv.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 0d3148bb..be413ae4 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -49,7 +49,7 @@ object SharkEnv extends LogHelper { def initWithSharkContext(jobName: String, master: String = System.getenv("MASTER")) : SharkContext = { if (sc != null) { - sc.stop + sc.stop() } sc = new SharkContext( @@ -64,7 +64,7 @@ object SharkEnv extends LogHelper { def initWithSharkContext(newSc: SharkContext): SharkContext = { if (sc != null) { - sc.stop + sc.stop() } sc = newSc From 87b494e70af10a9ebe25e1b26e69a6b516bf044c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 19 Oct 2013 07:47:52 -0700 Subject: [PATCH 137/331] Some style changes based on Reynold's feedback. --- src/main/scala/shark/SharkConfVars.scala | 2 +- .../execution/MemoryStoreSinkOperator.scala | 56 +++++++------------ .../scala/shark/execution/SharkDDLTask.scala | 45 ++++++--------- .../shark/execution/TableScanOperator.scala | 43 +++++++------- 4 files changed, 58 insertions(+), 88 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 8f502638..0b778976 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -49,7 +49,7 @@ object SharkConfVars { // Class name of the default cache policy used to manage partition evictions for cached, // Hive-partitioned tables. val CACHE_POLICY = new ConfVar( - "shark.cache.partition.cachePolicy.class", "shark.memstore2.LRUCachePolicy") + "shark.partition.cachePolicy.default", "shark.memstore2.LRUCachePolicy") // Maximum size - in terms of the number of objects - of the cache specified by the // "shark.cache.partition.cachePolicy" property above. diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index ffeb372b..3ef4958c 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -98,27 +98,25 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Put all rows of the table into a set of TablePartition's. Each partition contains // only one TablePartition object. - var outputRDD: RDD[TablePartition] = inputRdd.mapPartitionsWithIndex { - case(partitionIndex, iter) => { - op.initializeOnSlave() - val serde = new ColumnarSerDe - serde.initialize(op.localHconf, op.localHiveOp.getConf.getTableInfo.getProperties) - - // Serialize each row into the builder object. - // ColumnarSerDe will return a TablePartitionBuilder. - var builder: Writable = null - iter.foreach { row => - builder = serde.serialize(row.asInstanceOf[AnyRef], op.objectInspector) - } + var outputRDD: RDD[TablePartition] = inputRdd.mapPartitionsWithIndex { case (part, iter) => + op.initializeOnSlave() + val serde = new ColumnarSerDe + serde.initialize(op.localHconf, op.localHiveOp.getConf.getTableInfo.getProperties) + + // Serialize each row into the builder object. + // ColumnarSerDe will return a TablePartitionBuilder. + var builder: Writable = null + iter.foreach { row => + builder = serde.serialize(row.asInstanceOf[AnyRef], op.objectInspector) + } - if (builder != null) { - statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) - Iterator(builder.asInstanceOf[TablePartitionBuilder].build) - } else { - // Empty partition. - statsAcc += Tuple2(partitionIndex, new TablePartitionStats(Array(), 0)) - Iterator(new TablePartition(0, Array())) - } + if (builder != null) { + statsAcc += Tuple2(part, builder.asInstanceOf[TablePartitionBuilder].stats) + Iterator(builder.asInstanceOf[TablePartitionBuilder].build) + } else { + // Empty partition. + statsAcc += Tuple2(part, new TablePartitionStats(Array(), 0)) + Iterator(new TablePartition(0, Array())) } } @@ -133,10 +131,10 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Put the table in Tachyon. op.logInfo("Putting RDD for %s in Tachyon".format(tableName)) tachyonWriter.createTable(ByteBuffer.allocate(0)) - outputRDD = outputRDD.mapPartitionsWithIndex { case(partitionIndex, iter) => + outputRDD = outputRDD.mapPartitionsWithIndex { case(part, iter) => val partition = iter.next() partition.toTachyon.zipWithIndex.foreach { case(buf, column) => - tachyonWriter.writeColumnPartition(column, partitionIndex, buf) + tachyonWriter.writeColumnPartition(column, part, buf) } Iterator(partition) } @@ -202,20 +200,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { memoryTable.tableRDD = outputRDD } - // Report remaining memory. - /* Commented out for now waiting for the reporting code to make into Spark. - val remainingMems: Map[String, (Long, Long)] = SharkEnv.sc.getSlavesMemoryStatus - remainingMems.foreach { case(slave, mem) => - println("%s: %s / %s".format( - slave, - Utils.memoryBytesToString(mem._2), - Utils.memoryBytesToString(mem._1))) - } - println("Summary: %s / %s".format( - Utils.memoryBytesToString(remainingMems.map(_._2._2).sum), - Utils.memoryBytesToString(remainingMems.map(_._2._1).sum))) - */ - // TODO(harvey): Get this to work for Hive-partitioned tables. It should be a simple // 'tableName' + 'hivePartitionKey' concatentation. Though whether stats should belong in // memstore2.Table should be considered... diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 8d615e4e..e03774c3 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -52,18 +52,10 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] override def execute(driverContext: DriverContext): Int = { work.ddlDesc match { - case creatTblDesc: CreateTableDesc => { - createTable(creatTblDesc, work.cacheMode) - } - case addPartitionDesc: AddPartitionDesc => { - addPartition(addPartitionDesc) - } - case dropTableDesc: DropTableDesc => { - dropTableOrPartition(dropTableDesc) - } - case alterTableDesc: AlterTableDesc => { - alterTable(alterTableDesc) - } + case creatTblDesc: CreateTableDesc => createTable(creatTblDesc, work.cacheMode) + case addPartitionDesc: AddPartitionDesc => addPartition(addPartitionDesc) + case dropTableDesc: DropTableDesc => dropTableOrPartition(dropTableDesc) + case alterTableDesc: AlterTableDesc => alterTable(alterTableDesc) case _ => { throw new UnsupportedOperationException( "Shark does not require a Shark DDL task for: " + work.ddlDesc.getClass.getName) @@ -76,9 +68,7 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } /** Handles a CREATE TABLE or CTAS. */ - def createTable( - createTblDesc: CreateTableDesc, - cacheMode: CacheType.CacheType) { + def createTable(createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps @@ -108,12 +98,11 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] // Find the set of partition column values that specifies the partition being added. val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionColumns: Seq[String] = hiveTable.getPartCols.map(_.getName) - val partitionColumnToValue: JavaMap[String, String] = addPartitionDesc.getPartSpec + val partCols: Seq[String] = hiveTable.getPartCols.map(_.getName) + val partColToValue: JavaMap[String, String] = addPartitionDesc.getPartSpec // String format for partition key: 'col1=value1/col2=value2/...' - val partitionKeyStr: String = MemoryMetadataManager.makeHivePartitionKeyStr( - partitionColumns, partitionColumnToValue) - partitionedTable.putPartition(partitionKeyStr, new EmptyRDD(SharkEnv.sc)) + val partKeyStr: String = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partColToValue) + partitionedTable.putPartition(partKeyStr, new EmptyRDD(SharkEnv.sc)) } /** @@ -123,23 +112,21 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] def dropTableOrPartition(dropTableDesc: DropTableDesc) { val tableName = dropTableDesc.getTableName val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); - val partitionSpecs: JavaList[PartitionSpec] = dropTableDesc.getPartSpecs + val partSpecs: JavaList[PartitionSpec] = dropTableDesc.getPartSpecs - if (partitionSpecs == null) { + if (partSpecs == null) { // The command is a true DROP TABLE. SharkEnv.dropTable(tableName) } else { // The command is an ALTER TABLE DROP PARTITION val partitionedTable = getPartitionedTableWithAssertions(tableName) // Find the set of partition column values that specifies the partition being dropped. - val partitionColumns: Seq[String] = hiveTable.getPartCols.map(_.getName) - for (partitionSpec <- partitionSpecs) { - val partitionColumnToValue: JavaMap[String, String] = - partitionSpec.getPartSpecWithoutOperator + val partCols: Seq[String] = hiveTable.getPartCols.map(_.getName) + for (partSpec <- partSpecs) { + val partColToValue: JavaMap[String, String] = partSpec.getPartSpecWithoutOperator // String format for partition key: 'col1=value1/col2=value2/...' - val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr( - partitionColumns, partitionColumnToValue) - partitionedTable.removePartition(partitionKeyStr) + val partKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partColToValue) + partitionedTable.removePartition(partKeyStr) } } } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 7e17199e..47e7f57c 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectIns import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.io.Writable -import org.apache.spark.rdd.{PartitionPruningRDD, RDD, UnionRDD} +import org.apache.spark.rdd.{EmptyRDD, PartitionPruningRDD, RDD, UnionRDD} import shark.{SharkConfVars, SharkEnv, Utils} import shark.api.QueryExecutionException @@ -255,27 +255,26 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO * partition key specifications. */ private def makeCachedPartitionRDD(tableKey: String, partitions: Array[Partition]): RDD[_] = { - val hivePartitionRDDSeq = partitions.map { partition => + val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) // Get partition field info val partSpec = partDesc.getPartSpec() val partProps = partDesc.getProperties() val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - // Partitioning keys are delimited by "/" - val partColumns = partColsDelimited.trim().split("/").toSeq - // 'partValues[i]' contains the value for the partitioning column at 'partColumns[i]'. - val partValues = partColumns.map { key => - if (partSpec == null) { - new String - } else { - new String(partSpec.get(key)) - } - }.toArray - val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partColumns, partSpec) + // Partitioning columns are delimited by "/" + val partCols = partColsDelimited.trim().split("/").toSeq + // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. + val partValues = if (partSpec == null) { + Arrays.fill(partCols.size)(new String) + } else { + partCols.map(new String(partSpec.get(_))).toArray + } + + val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) val hivePartitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableKey).get val hivePartitionRDD = hivePartitionedTable.getPartition(partitionKeyStr) - + hivePartitionRDD.get.mapPartitions { iter => if (iter.hasNext) { // Map each tuple to a row object @@ -291,10 +290,10 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO } } } - if (hivePartitionRDDSeq.size > 0) { - new UnionRDD(hivePartitionRDDSeq.head.context, hivePartitionRDDSeq) + if (hivePartitionRDDs.size > 0) { + new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs) } else { - SharkEnv.sc.makeRDD(Seq[Object]()) + new EmptyRDD[Object](SharkEnv.sc) } } @@ -325,16 +324,16 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO val partSpec = partDesc.getPartSpec() val partProps = partDesc.getProperties() - val partCols = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) // Partitioning keys are delimited by "/" - val partKeys = partCols.trim().split("/") - // 'partValues[i]' contains the value for the partitioning key at 'partKeys[i]'. + val partCols = partCols.trim().split("/") + // 'partValues[i]' contains the value for the partitioning column at 'partKeys[i]'. val partValues = new ArrayList[String] - partKeys.foreach { key => + partCols.foreach { col => if (partSpec == null) { partValues.add(new String) } else { - partValues.add(new String(partSpec.get(key))) + partValues.add(new String(partSpec.get(col))) } } From 82990935d4ca90387240eac18660912d22659bdf Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 19 Oct 2013 08:30:31 -0700 Subject: [PATCH 138/331] Minor cleanup. --- .../scala/shark/execution/SharkDDLTask.scala | 2 ++ .../shark/execution/TableScanOperator.scala | 19 ++++++++----------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index e03774c3..ccd73770 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -27,6 +27,8 @@ import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.plan.api.StageType +import org.apache.spark.rdd.EmptyRDD + import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index accaf13b..ee7a85a0 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -301,9 +301,9 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO val partCols = partColsDelimited.trim().split("/").toSeq // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. val partValues = if (partSpec == null) { - Arrays.fill(partCols.size)(new String) + Array.fill(partCols.size)(new String) } else { - partCols.map(new String(partSpec.get(_))).toArray + partCols.map(col => new String(partSpec.get(col))).toArray } val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) @@ -365,15 +365,12 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) // Partitioning keys are delimited by "/" - val partCols = partCols.trim().split("/") - // 'partValues[i]' contains the value for the partitioning column at 'partKeys[i]'. - val partValues = new ArrayList[String] - partCols.foreach { col => - if (partSpec == null) { - partValues.add(new String) - } else { - partValues.add(new String(partSpec.get(col))) - } + val partCols = partColsDelimited.trim().split("/") + // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. + val partValues = if (partSpec == null) { + Array.fill(partCols.size)(new String) + } else { + partCols.map(col => new String(partSpec.get(col))).toArray } val rowWithPartArr = new Array[Object](2) From f2b9c17853bbbc891516b863174eba347b983a23 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 19 Oct 2013 08:52:00 -0700 Subject: [PATCH 139/331] Clarify comment for RDDUtils#getStorageLevelOfRDDs() --- src/main/scala/shark/execution/RDDUtils.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 1acbdefe..5a56443c 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -48,7 +48,10 @@ object RDDUtils { /** * Returns the storage level of a sequence of RDDs, interpreted as the storage level of the first - * RDD in the sequence that persisted in memory or disk. + * RDD in the sequence that is persisted in memory or on disk. This works because for Shark's use + * case, all RDDs for a non-partitioned table should have the same storage level. An RDD for a + * partitioned table could be StorageLevel.NONE if it was unpersisted by the partition eviction + * policy. * * @param rdds The sequence of RDDs to find the StorageLevel of. */ From 0c581a912747882004b4f2952bd9ebd0b7196b66 Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Wed, 18 Sep 2013 21:27:29 -0700 Subject: [PATCH 140/331] scalastyle checks for sbt. All warnings for now. Execute using 'sbt scalastyle' Conflicts: project/SharkBuild.scala --- project/SharkBuild.scala | 4 +- project/plugins.sbt | 4 ++ scalastyle-config.xml | 118 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 124 insertions(+), 2 deletions(-) create mode 100644 scalastyle-config.xml diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 32154764..c9f32bb9 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -128,8 +128,8 @@ object SharkBuild extends Build { "net.java.dev.jets3t" % "jets3t" % "0.7.1", "com.novocode" % "junit-interface" % "0.8" % "test") ++ (if (YARN_ENABLED) Some("org.apache.spark" %% "spark-yarn" % SPARK_VERSION) else None).toSeq ++ - (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm) ) else None).toSeq - ) + (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm)) else None).toSeq + ) ++ org.scalastyle.sbt.ScalastylePlugin.Settings def assemblyProjSettings = Seq( jarName in assembly <<= version map { v => "shark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" } diff --git a/project/plugins.sbt b/project/plugins.sbt index d06b220d..5b2a7785 100755 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -15,6 +15,8 @@ addSbtPlugin("org.ensime" % "ensime-sbt-cmd" % "0.1.1") +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.3.2") + addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.4.0") @@ -24,3 +26,5 @@ addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") resolvers += Resolver.url( "sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) + +resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/" diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 00000000..a65482e0 --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,118 @@ + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From a420119d497c927955305c2cf8d9ace3cd2c29bb Mon Sep 17 00:00:00 2001 From: Nandu Jayakumar Date: Fri, 4 Oct 2013 07:50:51 -0700 Subject: [PATCH 141/331] Warnings now errors. Disabled cyclomatic complexity --- scalastyle-config.xml | 83 +++++++++++++++++++++++-------------------- 1 file changed, 45 insertions(+), 38 deletions(-) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index a65482e0..d3b75788 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -1,12 +1,19 @@ + + + + + + + Scalastyle standard configuration - - + + - + - - - - + + + + - + - + - + - - + + - + - + - - - - - - - - - + + + + + + + + + - + - - - - - - - - + + + + + + + + - + - + - + - - - + + + From 45b018ac9fecc4a596d358a1e500f6f26e415097 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 22 Oct 2013 20:53:34 -0700 Subject: [PATCH 142/331] Catch exceptions thrown by threads launched in ScriptOperator. --- .../shark/execution/ScriptOperator.scala | 73 ++++++++++++++++--- 1 file changed, 61 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 73c04f08..8066b2f5 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -17,26 +17,26 @@ package shark.execution -import java.io.{File, InputStream} +import java.io.{File, InputStream, IOException} +import java.lang.Thread.UncaughtExceptionHandler import java.util.{Arrays, Properties} - import scala.collection.JavaConversions._ import scala.io.Source import scala.reflect.BeanProperty - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{ScriptOperator => HiveScriptOperator} import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter, ScriptOperatorHelper} +import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.plan.ScriptDesc import org.apache.hadoop.hive.serde2.{Serializer, Deserializer} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.io.{BytesWritable, Writable} - import org.apache.spark.{OneToOneDependency, SparkEnv, SparkFiles} import org.apache.spark.rdd.RDD - +import org.apache.spark.TaskContext import shark.execution.serialization.OperatorSerializationWrapper +import shark.LogHelper /** @@ -67,7 +67,7 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { logDebug("Using %s and %s".format(outRecordReaderClass, inRecordWriterClass)) // Deserialize the output from script back to what Hive understands. - inputRdd.mapPartitions { part => + inputRdd.mapPartitionsWithContext { (context, part) => op.initializeOnSlave() // Serialize the data so it is recognizable by the script. @@ -98,27 +98,34 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { val sparkEnv = SparkEnv.get // Start a thread to print the process's stderr to ours - new Thread("stderr reader for " + command) { + val errorReaderThread = new Thread("stderr reader for " + command) { override def run() { - for(line <- Source.fromInputStream(proc.getErrorStream).getLines) { + for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { System.err.println(line) } } - }.start() + } + errorReaderThread.setUncaughtExceptionHandler( + new ScriptOperator.ScriptExceptionHandler(op, context)) + errorReaderThread.start() // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for " + command) { + val inputWriterThread = new Thread("stdin writer for " + command) { override def run() { // Set the thread local SparkEnv. SparkEnv.set(sparkEnv) val recordWriter = inRecordWriterClass.newInstance recordWriter.initialize(proc.getOutputStream, op.localHconf) - for(elem <- iter) { + + for (elem <- iter) { recordWriter.write(elem) } recordWriter.close() } - }.start() + } + inputWriterThread.setUncaughtExceptionHandler( + new ScriptOperator.ScriptExceptionHandler(op, context)) + inputWriterThread.start() // Return an iterator that reads outputs from RecordReader. Use our own // BinaryRecordReader if necessary because Hive's has a bug (see below). @@ -218,6 +225,9 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { } } + def allowPartialConsumption: Boolean = + HiveConf.getBoolVar(localHconf, HiveConf.ConfVars.ALLOWPARTIALCONSUMP) + def serializeForScript[T](iter: Iterator[T]): Iterator[Writable] = iter.map { row => scriptInputSerializer.serialize(row, objectInspector) } @@ -227,6 +237,45 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { object ScriptOperator { + /** + * A general exception handler to attach to child threads used to feed input rows and forward + * errors to the parent thread during ScriptOperator#execute(). + * If partial query consumption is not allowed (see HiveConf.Confvars.ALLOWPARTIALCONSUMP), then + * exceptions from child threads are caught by the handler and re-thrown by the parent thread + * through an on-task-completion callback registered with the Spark TaskContext. The task will be + * marked "failed" and the exception will be propagated to the master/CLI. + */ + class ScriptExceptionHandler( + serializedScriptOp: OperatorSerializationWrapper[ScriptOperator], context: TaskContext) + extends UncaughtExceptionHandler + with LogHelper { + + override def uncaughtException(thread: Thread, throwable: Throwable) { + throwable match { + case ioe: IOException => { + // Check whether the IOException should be re-thrown by the parent thread. + if (serializedScriptOp.allowPartialConsumption) { + logWarning("Error while executing script. Ignoring %s" + .format(ioe.getMessage)) + } else { + val onCompleteCallback = () => { + logWarning("Error during script execution. Set %s=true to ignore thrown IOExceptions." + .format(HiveConf.ConfVars.ALLOWPARTIALCONSUMP.toString)) + throw ioe + } + context.addOnCompleteCallback(onCompleteCallback) + } + } + case _ => { + // Throw any other Exceptions or Errors. + val onCompleteCallback = () => throw throwable + context.addOnCompleteCallback(onCompleteCallback) + } + } + } + + } + /** * An iterator that wraps around a Hive RecordReader. */ From 488930f5187264d094810f06f33b5b5a2fde230a Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 22 Oct 2013 23:08:40 -0700 Subject: [PATCH 143/331] Handle SparkHadoopWriter being made non-pubic. Added a simple CreateJobID() nested function --- .../scala/shark/execution/FileSinkOperator.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index 118a18a0..b990e1bb 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -17,6 +17,9 @@ package shark.execution +import java.text.SimpleDateFormat +import java.util.Date + import scala.reflect.BeanProperty import org.apache.hadoop.fs.FileSystem @@ -25,7 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} import org.apache.hadoop.hive.ql.exec.JobCloseFeedBack import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.hadoop.mapred.TaskID +import org.apache.hadoop.mapred.{JobID, TaskID} import org.apache.hadoop.mapred.TaskAttemptID import org.apache.hadoop.mapred.SparkHadoopWriter @@ -51,9 +54,15 @@ class FileSinkOperator extends TerminalOperator with Serializable { } def setConfParams(conf: HiveConf, context: TaskContext) { + def createJobID(time: Date, id: Int): JobID = { + val formatter = new SimpleDateFormat("yyyyMMddHHmm") + val jobtrackerID = formatter.format(new Date()) + return new JobID(jobtrackerID, id) + } + val jobID = context.stageId val splitID = context.splitId - val jID = SparkHadoopWriter.createJobID(now, jobID) + val jID = createJobID(now, jobID) val taID = new TaskAttemptID(new TaskID(jID, true, splitID), 0) conf.set("mapred.job.id", jID.toString) conf.set("mapred.tip.id", taID.getTaskID.toString) From d8c98c9a6290af673abedc8103e6286deedbb398 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 22 Oct 2013 23:44:04 -0700 Subject: [PATCH 144/331] Synchronize calls to TaskContext#addOnCompleteCallback() in ScriptOperator. --- .../shark/execution/ScriptOperator.scala | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 8066b2f5..336c4459 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -97,6 +97,9 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { // Get the thread local SparkEnv so we can pass it into the new thread. val sparkEnv = SparkEnv.get + // If true, exceptions thrown by child threads will be ignored. + val allowPartialConsumption = op.allowPartialConsumption + // Start a thread to print the process's stderr to ours val errorReaderThread = new Thread("stderr reader for " + command) { override def run() { @@ -106,7 +109,7 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { } } errorReaderThread.setUncaughtExceptionHandler( - new ScriptOperator.ScriptExceptionHandler(op, context)) + new ScriptOperator.ScriptExceptionHandler(allowPartialConsumption, context)) errorReaderThread.start() // Start a thread to feed the process input from our parent's iterator @@ -124,7 +127,7 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { } } inputWriterThread.setUncaughtExceptionHandler( - new ScriptOperator.ScriptExceptionHandler(op, context)) + new ScriptOperator.ScriptExceptionHandler(allowPartialConsumption, context)) inputWriterThread.start() // Return an iterator that reads outputs from RecordReader. Use our own @@ -245,8 +248,7 @@ object ScriptOperator { * through an on-task-completion callback registered with the Spark TaskContext. The task will be * marked "failed" and the exception will be propagated to the master/CLI. */ - class ScriptExceptionHandler( - serializedScriptOp: OperatorSerializationWrapper[ScriptOperator], context: TaskContext) + class ScriptExceptionHandler(allowPartialConsumption: Boolean, context: TaskContext) extends UncaughtExceptionHandler with LogHelper { @@ -254,7 +256,7 @@ object ScriptOperator { throwable match { case ioe: IOException => { // Check whether the IOException should be re-thrown by the parent thread. - if (serializedScriptOp.allowPartialConsumption) { + if (allowPartialConsumption) { logWarning("Error while executing script. Ignoring %s" .format(ioe.getMessage)) } else { @@ -263,17 +265,20 @@ object ScriptOperator { .format(HiveConf.ConfVars.ALLOWPARTIALCONSUMP.toString)) throw ioe } - context.addOnCompleteCallback(onCompleteCallback) + context.synchronized { + context.addOnCompleteCallback(onCompleteCallback) + } } } case _ => { // Throw any other Exceptions or Errors. val onCompleteCallback = () => throw throwable - context.addOnCompleteCallback(onCompleteCallback) + context.synchronized { + context.addOnCompleteCallback(onCompleteCallback) + } } } } - } /** From c4b6d50d8c41866dc0a43bdf708c2dba8964a0d0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 00:19:17 -0700 Subject: [PATCH 145/331] Fix some ScriptOperator spacing --- src/main/scala/shark/execution/ScriptOperator.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 336c4459..69eb1c09 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -20,9 +20,11 @@ package shark.execution import java.io.{File, InputStream, IOException} import java.lang.Thread.UncaughtExceptionHandler import java.util.{Arrays, Properties} + import scala.collection.JavaConversions._ import scala.io.Source import scala.reflect.BeanProperty + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{ScriptOperator => HiveScriptOperator} @@ -32,9 +34,11 @@ import org.apache.hadoop.hive.ql.plan.ScriptDesc import org.apache.hadoop.hive.serde2.{Serializer, Deserializer} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.io.{BytesWritable, Writable} + import org.apache.spark.{OneToOneDependency, SparkEnv, SparkFiles} import org.apache.spark.rdd.RDD import org.apache.spark.TaskContext + import shark.execution.serialization.OperatorSerializationWrapper import shark.LogHelper @@ -119,7 +123,6 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { SparkEnv.set(sparkEnv) val recordWriter = inRecordWriterClass.newInstance recordWriter.initialize(proc.getOutputStream, op.localHconf) - for (elem <- iter) { recordWriter.write(elem) } @@ -253,6 +256,7 @@ object ScriptOperator { with LogHelper { override def uncaughtException(thread: Thread, throwable: Throwable) { + Thread.sleep(7000) throwable match { case ioe: IOException => { // Check whether the IOException should be re-thrown by the parent thread. @@ -265,6 +269,7 @@ object ScriptOperator { .format(HiveConf.ConfVars.ALLOWPARTIALCONSUMP.toString)) throw ioe } + println("thread finished...") context.synchronized { context.addOnCompleteCallback(onCompleteCallback) } From 9b949b38356da44965dda49ac1bcffb3a1680cd1 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 04:09:43 -0700 Subject: [PATCH 146/331] Add FIFO, cache-all policies. Includes better support for tracking stats and the addition of a basic, abstract LinkedHashMapBasedPolicy that's used for FIFO and LRU policies. --- src/main/scala/shark/SharkConfVars.scala | 7 +- .../scala/shark/memstore2/CachePolicy.scala | 41 ++++-- .../shark/memstore2/LRUCachePolicy.scala | 94 -------------- .../memstore2/LinkedMapBasedPolicy.scala | 117 ++++++++++++++++++ .../memstore2/MemoryMetadataManager.scala | 7 +- .../memstore2/PartitionedMemoryTable.scala | 20 ++- src/test/scala/shark/SQLSuite.scala | 43 ++----- 7 files changed, 169 insertions(+), 160 deletions(-) delete mode 100644 src/main/scala/shark/memstore2/LRUCachePolicy.scala create mode 100644 src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 0b778976..1ebee61e 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -49,17 +49,12 @@ object SharkConfVars { // Class name of the default cache policy used to manage partition evictions for cached, // Hive-partitioned tables. val CACHE_POLICY = new ConfVar( - "shark.partition.cachePolicy.default", "shark.memstore2.LRUCachePolicy") + "shark.partition.cachePolicy.default", "shark.memstore2.CacheAllPolicy") // Maximum size - in terms of the number of objects - of the cache specified by the // "shark.cache.partition.cachePolicy" property above. val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.partition.cachePolicy.maxSize", "10") - // Whether the cache specified by "shark.cache.partition.cachePolicy" above should record - // statistics (e.g., hit rate). - val SHOULD_RECORD_PARTITION_CACHE_STATS = new ConfVar( - "shark.cache.partition.cachePolicy.shouldRecordStats", false) - // If true, then cache any table whose name ends in "_cached". val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index b3ebb4e1..bd07352d 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -17,29 +17,34 @@ package shark.memstore2 +import java.util.concurrent.ConcurrentHashMap -trait CachePolicy[K, V] { +import scala.collection.JavaConversions._ + +import org.apache.spark.rdd.RDD - protected var maxSize: Long = _ + +trait CachePolicy[K, V] { protected var loadFunc: (K => V) = _ protected var evictionFunc: (K, V) => Unit = _ + protected var maxSize: Int = -1 + def initialize( - maxSize: Long, + maxSize: Int, loadFunc: (K => V), - evictionFunc: (K, V) => Unit, - shouldRecordStats: Boolean + evictionFunc: (K, V) => Unit ): Unit = { this.maxSize = maxSize this.loadFunc = loadFunc this.evictionFunc = evictionFunc } - + def notifyPut(key: K, value: V): Unit - def notifyRemove(key: K, value: V): Unit + def notifyRemove(key: K): Unit def notifyGet(key: K): Unit @@ -47,7 +52,25 @@ trait CachePolicy[K, V] { def getMaxSize = maxSize - def getHitRate: Option[Double] = None + def getHitRate: Double + + def getEvictionCount: Long +} + + +class CacheAllPolicy[K, V] extends CachePolicy[K, V] { + + var keyToRdds = new ConcurrentHashMap[K, V]() + + override def notifyPut(key: K, value: V) = keyToRdds.put(key, value) + + override def notifyRemove(key: K) = keyToRdds.remove(key) + + override def notifyGet(key: K) = Unit + + override def getKeysOfCachedEntries: Seq[K] = keyToRdds.keySet.toSeq + + override def getHitRate: Double = 1.0 - def getEvictionCount: Option[Long] = None + override def getEvictionCount: Long = 0L } diff --git a/src/main/scala/shark/memstore2/LRUCachePolicy.scala b/src/main/scala/shark/memstore2/LRUCachePolicy.scala deleted file mode 100644 index c0d808f0..00000000 --- a/src/main/scala/shark/memstore2/LRUCachePolicy.scala +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import com.google.common.cache._ - -import scala.collection.JavaConversions._ - - -class LRUCachePolicy[K <: AnyRef, V <: AnyRef] extends CachePolicy[K, V] { - - var isInitialized = false - var hasRecordedStats = false - var cache: LoadingCache[K, V] = _ - var cacheStats: Option[CacheStats] = None - - override def initialize( - maxSize: Long, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit, - shouldRecordStats: Boolean - ): Unit = { - super.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) - - var builder = CacheBuilder.newBuilder().maximumSize(maxSize) - if (shouldRecordStats) { - builder.recordStats() - hasRecordedStats = true - } - - // Automatically executed by 'cache' on the entry being evicted. - val removalListener = - new RemovalListener[K, V] { - def onRemoval(removal: RemovalNotification[K, V]): Unit = { - evictionFunc(removal.getKey, removal.getValue) - } - } - // Automatically executed by 'cache' on the entry being loaded. - val cacheLoader = - new CacheLoader[K, V] { - def load(key: K): V = loadFunc(key) - } - - cache = builder - .removalListener(removalListener) - .build(cacheLoader) - isInitialized = true - } - - override def notifyPut(key: K, value: V): Unit = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - cache.put(key, value) - } - - override def notifyRemove(key: K, value: V): Unit = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - cache.invalidate(key, value) - } - - override def notifyGet(key: K): Unit = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - cache.get(key) - } - - override def getKeysOfCachedEntries: Seq[K] = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - return cache.asMap.keySet.toSeq - } - - override def getHitRate(): Option[Double] = { - val hitRate = if (hasRecordedStats) Some(cache.stats.hitRate) else None - return hitRate - } - - override def getEvictionCount(): Option[Long] = { - val evictionCount = if (hasRecordedStats) Some(cache.stats.evictionCount) else None - return evictionCount - } -} diff --git a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala new file mode 100644 index 00000000..c5832860 --- /dev/null +++ b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.util.LinkedHashMap +import java.util.Map.Entry + +import scala.collection.JavaConversions._ + +class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { + + override def initialize( + maxSize: Int, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit + ): Unit = { + super.initialize(maxSize, loadFunc, evictionFunc) + cache = new LinkedMapCache(true /* accessOrder */) + } + +} + + +class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { + + override def initialize( + maxSize: Int, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit + ): Unit = { + super.initialize(maxSize, loadFunc, evictionFunc) + cache = new LinkedMapCache() + } + +} + + +sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { + + class LinkedMapCache(val accessOrder: Boolean = false) + extends LinkedHashMap[K, V](maxSize, 0.75F, accessOrder) { + + override def removeEldestEntry(eldest: Entry[K, V]): Boolean = { + evictionFunc(eldest.getKey, eldest.getValue) + evictionCount += 1 + return (size() > maxSize) + } + } + + var cache: LinkedMapCache = _ + var isInitialized = false + var hitCount: Long = 0L + var missCount: Long = 0L + var evictionCount: Long = 0L + + override def initialize( + maxSize: Int, + loadFunc: (K => V), + evictionFunc: (K, V) => Unit + ): Unit = { + super.initialize(maxSize, loadFunc, evictionFunc) + isInitialized = true + } + + override def notifyPut(key: K, value: V): Unit = { + assert(isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { cache.put(key, value) } + } + + override def notifyRemove(key: K): Unit = { + assert(isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { cache.remove(key) } + } + + override def notifyGet(key: K): Unit = { + assert(isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { + if (cache.contains(key)) { + cache.get(key) + hitCount += 1L + } else { + val retrievedValue = loadFunc(key) + cache.put(key, retrievedValue) + missCount += 1L + } + } + } + + override def getKeysOfCachedEntries: Seq[K] = { + assert(isInitialized, "Must initialize() LRUCachePolicy.") + return cache.keySet.toSeq + } + + override def getHitRate: Double = { + val requestCount = missCount + hitCount + val hitRate = if (requestCount == 0L) 1.0 else (hitCount / requestCount) + return hitRate + } + + override def getEvictionCount = evictionCount + +} diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 2a9ff05b..f32b97b7 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -76,11 +76,8 @@ class MemoryMetadataManager { SharkConfVars.CACHE_POLICY.defaultVal) val maxCacheSize = tblProps.getOrElse( SharkConfVars.MAX_PARTITION_CACHE_SIZE.varname, - SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toLong - val shouldRecordStats = tblProps.getOrElse( - SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.varname, - SharkConfVars.SHOULD_RECORD_PARTITION_CACHE_STATS.defaultVal).toBoolean - newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize, shouldRecordStats) + SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toInt + newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize) } _keyToTable.put(tableName.toLowerCase, newTable) diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 57c29761..41967057 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -60,13 +60,13 @@ class PartitionedMemoryTable( // can be set from the CLI: // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. // If 'None', then all partitions will be persisted in memory using the 'preferredStorageLevel'. - private var _cachePolicy: Option[CachePolicy[String, RDDValue]] = None + private var _cachePolicy: CachePolicy[String, RDDValue] = _ def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) def getPartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddValueOpt: Option[RDDValue] = _keyToPartitions.get(partitionKey) - if (rddValueOpt.isDefined && _cachePolicy.isDefined) _cachePolicy.get.notifyGet(partitionKey) + if (rddValueOpt.isDefined) _cachePolicy.notifyGet(partitionKey) return rddValueOpt.map(_.rdd) } @@ -87,24 +87,20 @@ class PartitionedMemoryTable( } else { val newRDDValue = new RDDValue(newRDD) _keyToPartitions.put(partitionKey, newRDDValue) - if (_cachePolicy.isDefined) _cachePolicy.get.notifyPut(partitionKey, newRDDValue) + _cachePolicy.notifyPut(partitionKey, newRDDValue) } return prevRDD } def removePartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) - if (rddRemoved.isDefined && _cachePolicy.isDefined) { - _cachePolicy.get.notifyRemove(partitionKey, rddRemoved.get) - } + if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey) return rddRemoved.map(_.rdd) } def setPartitionCachePolicy( cachePolicyStr: String, - maxSize: Long, - shouldRecordStats: Boolean - ) { + maxSize: Int) { val newPolicy = Class.forName(cachePolicyStr).newInstance .asInstanceOf[CachePolicy[String, RDDValue]] // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. @@ -119,11 +115,11 @@ class PartitionedMemoryTable( // The evitionFunc will unpersist the RDD. val evictionFunc: (String, RDDValue) => Unit = (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) - newPolicy.initialize(maxSize, loadFunc, evictionFunc, shouldRecordStats) - _cachePolicy = Some(newPolicy) + newPolicy.initialize(maxSize, loadFunc, evictionFunc) + _cachePolicy = newPolicy } - def cachePolicy: Option[CachePolicy[String, RDDValue]] = _cachePolicy + def cachePolicy: CachePolicy[String, RDDValue] = _cachePolicy /** Returns an immutable view of the String->RDD mapping to external callers */ def keyToPartitions: collection.immutable.Map[String, RDD[TablePartition]] = { diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index f6daa1a1..c50c7da7 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -108,8 +108,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { tableName: String, numPartitionsToCreate: Int, maxCacheSize: Int = 10, - cachePolicyClassName: String = "shark.memstore2.LRUCachePolicy", - shouldRecordStats: Boolean = false + cachePolicyClassName: String = "shark.memstore2.LRUCachePolicy" ): PartitionedMemoryTable = { sc.runSql("drop table if exists %s".format(tableName)) sc.runSql(""" @@ -118,13 +117,11 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { tblproperties('shark.cache' = 'true', 'shark.cache.partition.cachePolicy.maxSize' = '%d', 'shark.cache.partition.cachePolicy.class' = '%s', - 'shark.cache.storageLevel' = 'MEMORY_AND_DISK', - 'shark.cache.partition.cachePolicy.shouldRecordStats' = '%b') + 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') """.format( tableName, maxCacheSize, - cachePolicyClassName, - shouldRecordStats)) + cachePolicyClassName)) var partitionNum = 1 while (partitionNum <= numPartitionsToCreate) { sc.runSql("""insert into table %s partition(keypart = %d) @@ -515,7 +512,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { partitioned by (keypart int)""") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val cachePolicy = partitionedTable.cachePolicy.get + val cachePolicy = partitionedTable.cachePolicy assert(cachePolicy.isInstanceOf[shark.memstore2.LRUCachePolicy[_, _]]) } @@ -641,40 +638,18 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) } - test("LRU: record cache stats if user specifies it") { + test("LRU: record cache stats") { val tableName = "should_record_partition_cache_stats" val partitionedTable = createCachedPartitionedTable( tableName, 1 /* numPartitionsToCreate */, 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy", - true /* shouldRecordStats */) - val lruCachePolicy = partitionedTable.cachePolicy.get - val hitRate = lruCachePolicy.getHitRate - assert(hitRate.isDefined) - assert(hitRate.get == 1.0) - val evictionCount = lruCachePolicy.getEvictionCount - assert(evictionCount.isDefined) - assert(evictionCount.get == 0) - } - - test("LRU: cache stats are not recorded by default") { - sc.runSql("drop table if exists dont_record_partition_cache_stats") - sc.runSql(""" - create table dont_record_partition_cache_stats(key int, value string) - partitioned by (keypart int) - tblproperties( - 'shark.cache' = 'true', - 'shark.cache.partition.cachePolicy.class' = 'shark.memstore2.LRUCachePolicy') - """) - val tableName = "dont_record_partition_cache_stats" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - val lruCachePolicy = partitionedTable.cachePolicy.get + "shark.memstore2.LRUCachePolicy") + val lruCachePolicy = partitionedTable.cachePolicy val hitRate = lruCachePolicy.getHitRate - assert(hitRate.isEmpty) + assert(hitRate == 1.0) val evictionCount = lruCachePolicy.getEvictionCount - assert(evictionCount.isEmpty) + assert(evictionCount == 0) } ////////////////////////////////////////////////////////////////////////////// From faf5622df9300e709b744350080ab01f6dbe91b8 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 04:32:47 -0700 Subject: [PATCH 147/331] CachePolicy/LinkedMapBasedPolicy style changes. --- .../scala/shark/memstore2/CachePolicy.scala | 26 ++++---- .../memstore2/LinkedMapBasedPolicy.scala | 64 ++++++++++--------- .../memstore2/PartitionedMemoryTable.scala | 4 ++ 3 files changed, 51 insertions(+), 43 deletions(-) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index bd07352d..97202cf7 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -26,20 +26,20 @@ import org.apache.spark.rdd.RDD trait CachePolicy[K, V] { - protected var loadFunc: (K => V) = _ + protected var _loadFunc: (K => V) = _ - protected var evictionFunc: (K, V) => Unit = _ + protected var _evictionFunc: (K, V) => Unit = _ - protected var maxSize: Int = -1 + protected var _maxSize: Int = -1 def initialize( maxSize: Int, loadFunc: (K => V), evictionFunc: (K, V) => Unit ): Unit = { - this.maxSize = maxSize - this.loadFunc = loadFunc - this.evictionFunc = evictionFunc + _maxSize = maxSize + _loadFunc = loadFunc + _evictionFunc = evictionFunc } def notifyPut(key: K, value: V): Unit @@ -48,13 +48,13 @@ trait CachePolicy[K, V] { def notifyGet(key: K): Unit - def getKeysOfCachedEntries: Seq[K] + def keysOfCachedEntries: Seq[K] - def getMaxSize = maxSize + def maxSize: Int = _maxSize - def getHitRate: Double + def hitRate: Double - def getEvictionCount: Long + def evictionCount: Long } @@ -68,9 +68,9 @@ class CacheAllPolicy[K, V] extends CachePolicy[K, V] { override def notifyGet(key: K) = Unit - override def getKeysOfCachedEntries: Seq[K] = keyToRdds.keySet.toSeq + override def keysOfCachedEntries: Seq[K] = keyToRdds.keySet.toSeq - override def getHitRate: Double = 1.0 + override def hitRate = 1.0 - override def getEvictionCount: Long = 0L + override def evictionCount = 0L } diff --git a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala index c5832860..9ebee66b 100644 --- a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala +++ b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala @@ -30,7 +30,7 @@ class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { evictionFunc: (K, V) => Unit ): Unit = { super.initialize(maxSize, loadFunc, evictionFunc) - cache = new LinkedMapCache(true /* accessOrder */) + _cache = new LinkedMapCache(true /* accessOrder */) } } @@ -44,7 +44,7 @@ class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { evictionFunc: (K, V) => Unit ): Unit = { super.initialize(maxSize, loadFunc, evictionFunc) - cache = new LinkedMapCache() + _cache = new LinkedMapCache() } } @@ -52,21 +52,21 @@ class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { - class LinkedMapCache(val accessOrder: Boolean = false) + class LinkedMapCache(accessOrder: Boolean = false) extends LinkedHashMap[K, V](maxSize, 0.75F, accessOrder) { override def removeEldestEntry(eldest: Entry[K, V]): Boolean = { - evictionFunc(eldest.getKey, eldest.getValue) - evictionCount += 1 + _evictionFunc(eldest.getKey, eldest.getValue) + _evictionCount += 1 return (size() > maxSize) } } - var cache: LinkedMapCache = _ - var isInitialized = false - var hitCount: Long = 0L - var missCount: Long = 0L - var evictionCount: Long = 0L + protected var _cache: LinkedMapCache = _ + protected var _isInitialized = false + protected var _hitCount: Long = 0L + protected var _missCount: Long = 0L + protected var _evictionCount: Long = 0L override def initialize( maxSize: Int, @@ -74,44 +74,48 @@ sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { evictionFunc: (K, V) => Unit ): Unit = { super.initialize(maxSize, loadFunc, evictionFunc) - isInitialized = true + _isInitialized = true } override def notifyPut(key: K, value: V): Unit = { - assert(isInitialized, "Must initialize() %s.".format(this.getClass.getName)) - this.synchronized { cache.put(key, value) } + assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { _cache.put(key, value) } } override def notifyRemove(key: K): Unit = { - assert(isInitialized, "Must initialize() %s.".format(this.getClass.getName)) - this.synchronized { cache.remove(key) } + assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { _cache.remove(key) } } override def notifyGet(key: K): Unit = { - assert(isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) this.synchronized { - if (cache.contains(key)) { - cache.get(key) - hitCount += 1L + if (_cache.contains(key)) { + _cache.get(key) + _hitCount += 1L } else { - val retrievedValue = loadFunc(key) - cache.put(key, retrievedValue) - missCount += 1L + val retrievedValue = _loadFunc(key) + _cache.put(key, retrievedValue) + _missCount += 1L } } } - override def getKeysOfCachedEntries: Seq[K] = { - assert(isInitialized, "Must initialize() LRUCachePolicy.") - return cache.keySet.toSeq + override def keysOfCachedEntries: Seq[K] = { + assert(_isInitialized, "Must initialize() LRUCachePolicy.") + this.synchronized { + return _cache.keySet.toSeq + } } - override def getHitRate: Double = { - val requestCount = missCount + hitCount - val hitRate = if (requestCount == 0L) 1.0 else (hitCount / requestCount) - return hitRate + override def hitRate: Double = { + this.synchronized { + val requestCount = _missCount + _hitCount + val rate = if (requestCount == 0L) 1.0 else (_hitCount / requestCount) + return rate + } } - override def getEvictionCount = evictionCount + override def evictionCount = _evictionCount } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 41967057..5c28be8c 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -107,6 +107,10 @@ class PartitionedMemoryTable( val loadFunc: String => RDDValue = (partitionKey: String) => { val rddValue = _keyToPartitions.get(partitionKey).get + // RDDUtils#getStorageLevelOfRDD() will return StorageLevel.NONE only if all parent RDDs of + // a UnionRDD are not persisted. That function is used to check the RDD's storage level in + // order to avoid directly persisting an UnionRDD comprising RDDs already persisted in + // memory, which leads to duplicate data. if (RDDUtils.getStorageLevelOfRDD(rddValue.rdd) == StorageLevel.NONE) { rddValue.rdd.persist(preferredStorageLevel) } From 11d124aad9cf1c3d4a8fd36da622ad1237d8d417 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 06:37:30 -0700 Subject: [PATCH 148/331] Remove Table#getCurrentStorageLevel() --- src/main/scala/shark/memstore2/MemoryTable.scala | 1 - .../scala/shark/memstore2/PartitionedMemoryTable.scala | 8 -------- src/main/scala/shark/memstore2/Table.scala | 9 +-------- 3 files changed, 1 insertion(+), 17 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 1fb50ce0..020a112f 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -40,5 +40,4 @@ class MemoryTable( def tableRDD_= (rdd: RDD[TablePartition]) = _tableRDD = rdd - override def getCurrentStorageLevel: StorageLevel = RDDUtils.getStorageLevelOfRDD(tableRDD) } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 5c28be8c..0fad19a6 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -130,12 +130,4 @@ class PartitionedMemoryTable( return _keyToPartitions.mapValues(_.rdd).toMap } - /** - * Computes the current storage level for this table. See comments in - * RDDUtils#getStorageLevelOfRDDs() for how Shark interprets the storage level of a sequence of - * RDDs. - */ - override def getCurrentStorageLevel: StorageLevel = { - return RDDUtils.getStorageLevelOfRDDs(_keyToPartitions.values.map(_.rdd).toSeq) - } } diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index a6ba4b0d..5e9af782 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -34,11 +34,4 @@ import org.apache.spark.storage.StorageLevel private[shark] abstract class Table( var tableName: String, var cacheMode: CacheType.CacheType, - var preferredStorageLevel: StorageLevel - ) { - - /** - * Compute the current storage level of RDDs that back this table. - */ - def getCurrentStorageLevel: StorageLevel -} + var preferredStorageLevel: StorageLevel) From 3498a59a3fe98191c18947bb9af807de187071ef Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 06:40:58 -0700 Subject: [PATCH 149/331] Separate updatePartition() from putPartition() in PartitionedMemoryTable. --- .../execution/MemoryStoreSinkOperator.scala | 2 +- .../memstore2/PartitionedMemoryTable.scala | 28 +++++++++++-------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 3ef4958c..2d8de473 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -187,7 +187,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // An RDD for the Hive partition already exists, so update its metadata entry in // 'partitionedTable'. assert(outputRDD.isInstanceOf[UnionRDD[_]]) - partitionedTable.putPartition(hivePartitionKey, outputRDD, true /* isUpdate */) + partitionedTable.updatePartition(hivePartitionKey, outputRDD) } else { // This is a new Hive-partition. Add a new metadata entry in 'partitionedTable'. partitionedTable.putPartition(hivePartitionKey, outputRDD) diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 0fad19a6..cd48ad44 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -76,18 +76,24 @@ class PartitionedMemoryTable( isUpdate: Boolean = false): Option[RDD[TablePartition]] = { val rddValueOpt = _keyToPartitions.get(partitionKey) var prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) - if (isUpdate && rddValueOpt.isDefined) { - // This is an update of an old value, so update the RDDValue's 'rdd' entry. - // Don't notify the '_cachePolicy'. Assumes that getPartition() has already been called to - // obtain the value of the previous RDD, and that an RDD update refers to the RDD created from - // a transform or union. + val newRDDValue = new RDDValue(newRDD) + _keyToPartitions.put(partitionKey, newRDDValue) + _cachePolicy.notifyPut(partitionKey, newRDDValue) + return prevRDD + } + + def updatePartition( + partitionKey: String, + updatedRDD: RDD[TablePartition]): Option[RDD[TablePartition]] = { + val rddValueOpt = _keyToPartitions.get(partitionKey) + var prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) + if (rddValueOpt.isDefined) { + // This is an update of an old value, so update the RDDValue's `rdd` entry. + // Don't notify the `_cachePolicy`. Assumes that getPartition() has already been called to + // obtain the value of the previous RDD. + // An RDD update refers to the RDD created from a transform or union. val updatedRDDValue = rddValueOpt.get - updatedRDDValue.rdd = newRDD - updatedRDDValue - } else { - val newRDDValue = new RDDValue(newRDD) - _keyToPartitions.put(partitionKey, newRDDValue) - _cachePolicy.notifyPut(partitionKey, newRDDValue) + updatedRDDValue.rdd = updatedRDD } return prevRDD } From c98c3c73f8254b5437620fe772142f20f4a62941 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 06:43:36 -0700 Subject: [PATCH 150/331] Allow users to add arguments to CachePolicy table properties. e.g., "set shark.cache.policy=LRUCachePolicy(30)" --- src/main/scala/shark/SharkConfVars.scala | 7 +-- .../scala/shark/memstore2/CachePolicy.scala | 62 +++++++++++++------ .../memstore2/LinkedMapBasedPolicy.scala | 25 +++----- .../memstore2/MemoryMetadataManager.scala | 37 +++++------ .../memstore2/PartitionedMemoryTable.scala | 23 +++---- 5 files changed, 78 insertions(+), 76 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 1ebee61e..8da524ea 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -43,17 +43,14 @@ object SharkConfVars { // Default storage level for cached tables. val STORAGE_LEVEL = new ConfVar("shark.cache.storageLevel", "MEMORY_AND_DISK") - // True if a table's partitions should be evicted using a cache policy. - val SHOULD_USE_CACHE_POLICY = new ConfVar("shark.cache.partition.useCachePolicy", true) - // Class name of the default cache policy used to manage partition evictions for cached, // Hive-partitioned tables. val CACHE_POLICY = new ConfVar( - "shark.partition.cachePolicy.default", "shark.memstore2.CacheAllPolicy") + "shark.cache.policy", "shark.memstore2.CacheAllPolicy") // Maximum size - in terms of the number of objects - of the cache specified by the // "shark.cache.partition.cachePolicy" property above. - val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.partition.cachePolicy.maxSize", "10") + val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.policy.maxSize", "10") // If true, then cache any table whose name ends in "_cached". val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index 97202cf7..cfc42d05 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -24,6 +24,24 @@ import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD +class CacheAllPolicy[K, V] extends CachePolicy[K, V] { + + var keyToRdds = new ConcurrentHashMap[K, V]() + + override def notifyPut(key: K, value: V) = keyToRdds.put(key, value) + + override def notifyRemove(key: K) = keyToRdds.remove(key) + + override def notifyGet(key: K) = Unit + + override def keysOfCachedEntries: Seq[K] = keyToRdds.keySet.toSeq + + override def hitRate = 1.0 + + override def evictionCount = 0L +} + + trait CachePolicy[K, V] { protected var _loadFunc: (K => V) = _ @@ -32,12 +50,18 @@ trait CachePolicy[K, V] { protected var _maxSize: Int = -1 - def initialize( - maxSize: Int, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit - ): Unit = { - _maxSize = maxSize + def initializeWithUserSpecs(args: Array[String], fallbackMaxSize: Int) { + // By default, only initialize the `maxSize` from user specifications. + args.size match { + case 0 => _maxSize = fallbackMaxSize + case 1 => _maxSize = args.head.toInt + case _ => + throw new Exception("Accpted format: %s(maxSize: Int)".format(this.getClass.getName)) + } + } + + def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { + require(maxSize > 0, "Must specify a maxSize before initializing CachePolicy internals.") _loadFunc = loadFunc _evictionFunc = evictionFunc } @@ -52,25 +76,23 @@ trait CachePolicy[K, V] { def maxSize: Int = _maxSize + // TODO(harvey): Call this in Shark's handling of ALTER TABLE TBLPROPERTIES. + def maxSize_= (newMaxSize: Int) = _maxSize = newMaxSize + def hitRate: Double def evictionCount: Long } -class CacheAllPolicy[K, V] extends CachePolicy[K, V] { - - var keyToRdds = new ConcurrentHashMap[K, V]() - - override def notifyPut(key: K, value: V) = keyToRdds.put(key, value) - - override def notifyRemove(key: K) = keyToRdds.remove(key) - - override def notifyGet(key: K) = Unit - - override def keysOfCachedEntries: Seq[K] = keyToRdds.keySet.toSeq +object CachePolicy { - override def hitRate = 1.0 - - override def evictionCount = 0L + def instantiateWithUserSpecs[K, V](str: String, fallbackMaxSize: Int): CachePolicy[K, V] = { + val firstParenPos = str.indexOf('(') + val classStr = str.slice(0, firstParenPos) + val strArgs = str.substring(firstParenPos + 1, str.lastIndexOf(')')).split(',') + val policy = Class.forName(classStr).newInstance.asInstanceOf[CachePolicy[K, V]] + policy.initializeWithUserSpecs(strArgs, fallbackMaxSize) + return policy + } } diff --git a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala index 9ebee66b..4d1e17b5 100644 --- a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala +++ b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala @@ -22,14 +22,11 @@ import java.util.Map.Entry import scala.collection.JavaConversions._ + class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { - override def initialize( - maxSize: Int, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit - ): Unit = { - super.initialize(maxSize, loadFunc, evictionFunc) + override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { + super.initializeInternals(loadFunc, evictionFunc) _cache = new LinkedMapCache(true /* accessOrder */) } @@ -38,12 +35,8 @@ class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { - override def initialize( - maxSize: Int, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit - ): Unit = { - super.initialize(maxSize, loadFunc, evictionFunc) + override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { + super.initializeInternals(loadFunc, evictionFunc) _cache = new LinkedMapCache() } @@ -68,12 +61,8 @@ sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { protected var _missCount: Long = 0L protected var _evictionCount: Long = 0L - override def initialize( - maxSize: Int, - loadFunc: (K => V), - evictionFunc: (K, V) => Unit - ): Unit = { - super.initialize(maxSize, loadFunc, evictionFunc) + override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { + super.initializeInternals(loadFunc, evictionFunc) _isInitialized = true } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index f32b97b7..8c0e0382 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -66,19 +66,12 @@ class MemoryMetadataManager { ): PartitionedMemoryTable = { var newTable = new PartitionedMemoryTable( tableName.toLowerCase, cacheMode, preferredStorageLevel) - val shouldUseCachePolicy = tblProps.getOrElse( - SharkConfVars.SHOULD_USE_CACHE_POLICY.varname, - SharkConfVars.SHOULD_USE_CACHE_POLICY.defaultBoolVal.toString).toBoolean - if (shouldUseCachePolicy) { - // Determine the cache policy to use and read any user-specified cache settings. - val cachePolicyStr = tblProps.getOrElse( - SharkConfVars.CACHE_POLICY.varname, - SharkConfVars.CACHE_POLICY.defaultVal) - val maxCacheSize = tblProps.getOrElse( - SharkConfVars.MAX_PARTITION_CACHE_SIZE.varname, - SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toInt - newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize) - } + // Determine the cache policy to use and read any user-specified cache settings. + val cachePolicyStr = tblProps.getOrElse(SharkConfVars.CACHE_POLICY.varname, + SharkConfVars.CACHE_POLICY.defaultVal) + val maxCacheSize = tblProps.getOrElse(SharkConfVars.MAX_PARTITION_CACHE_SIZE.varname, + SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toInt + newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize) _keyToTable.put(tableName.toLowerCase, newTable) return newTable @@ -146,7 +139,11 @@ class MemoryMetadataManager { return tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) } - /** Find all keys that are strings. Used to drop tables after exiting. */ + /** + * Find all keys that are strings. Used to drop tables after exiting. + * + * TODO(harvey): Won't be needed after unifed views are added. + */ def getAllKeyStrings(): Seq[String] = { _keyToTable.keys.collect { case k: String => k } toSeq } @@ -174,16 +171,12 @@ object MemoryMetadataManager { /** * Return a representation of the partition key in the string format: - * 'col1=value1/col2=value2/...' + * 'col1=value1/col2=value2/.../colN=valueN' */ def makeHivePartitionKeyStr( - partitionColumns: Seq[String], - partitionColumnToValue: JavaMap[String, String]): String = { - var keyStr = "" - for (partitionColumn <- partitionColumns) { - keyStr += "%s=%s/".format(partitionColumn, partitionColumnToValue(partitionColumn)) - } - keyStr = keyStr.dropRight(1) + partitionCols: Seq[String], + partColToValue: JavaMap[String, String]): String = { + val keyStr = partitionCols.map(col => "%s=%s".format(col, partColToValue(col))).mkString("/") return keyStr } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index cd48ad44..6d72269a 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -43,10 +43,11 @@ class PartitionedMemoryTable( extends Table(tableName, cacheMode, preferredStorageLevel) { /** - * A simple, mutable wrapper for an RDD. The value entires for a single key in - * '_keyToPartitions' and '_cachePolicy' will reference the same RDDValue object. This is needed - * so that entries maintained by a CachePolicy's cache map, such as the LRUCachePolicy#cache map, - * can be updated. The values can't be RDDs, which are inherently immutable... + * A simple, mutable wrapper for an RDD. This is needed so that a entry maintained by a + * CachePolicy's underlying data structure, such as the LinkedHashMap for LRUCachePolicy, can be + * updated without causing an eviction. + * The value entires for a single key in + * `_keyToPartitions` and `_cachePolicy` will reference the same RDDValue object. */ class RDDValue(var rdd: RDD[TablePartition]) @@ -58,8 +59,8 @@ class PartitionedMemoryTable( // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: - // 'TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")'. - // If 'None', then all partitions will be persisted in memory using the 'preferredStorageLevel'. + // `TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")`. + // If 'None', then all partitions will be persisted in memory using the `preferredStorageLevel`. private var _cachePolicy: CachePolicy[String, RDDValue] = _ def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) @@ -106,9 +107,9 @@ class PartitionedMemoryTable( def setPartitionCachePolicy( cachePolicyStr: String, - maxSize: Int) { - val newPolicy = Class.forName(cachePolicyStr).newInstance - .asInstanceOf[CachePolicy[String, RDDValue]] + fallbackMaxSize: Int) { + val newPolicy = CachePolicy.instantiateWithUserSpecs[String, RDDValue]( + cachePolicyStr, fallbackMaxSize) // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. val loadFunc: String => RDDValue = (partitionKey: String) => { @@ -122,10 +123,10 @@ class PartitionedMemoryTable( } rddValue } - // The evitionFunc will unpersist the RDD. + // The evictionFunc will unpersist the RDD. val evictionFunc: (String, RDDValue) => Unit = (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) - newPolicy.initialize(maxSize, loadFunc, evictionFunc) + newPolicy.initializeInternals(loadFunc, evictionFunc) _cachePolicy = newPolicy } From a7c3ea1cb05a439ad7b9c770dc1d994a1b85275c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 08:13:15 -0700 Subject: [PATCH 151/331] Some bugs from refactoring... --- .../scala/shark/memstore2/CachePolicy.scala | 32 +++++++++++------ .../memstore2/LinkedMapBasedPolicy.scala | 23 ++++++++---- src/test/scala/shark/SQLSuite.scala | 36 ++++++++++++------- 3 files changed, 62 insertions(+), 29 deletions(-) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index cfc42d05..e8bdf48b 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -23,18 +23,21 @@ import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD - +/** + * A cache that never evicts entries. + */ class CacheAllPolicy[K, V] extends CachePolicy[K, V] { - var keyToRdds = new ConcurrentHashMap[K, V]() + // Track the entries in the cache, so that keysOfCachedEntries() returns a valid result. + var cache = new ConcurrentHashMap[K, V]() - override def notifyPut(key: K, value: V) = keyToRdds.put(key, value) + override def notifyPut(key: K, value: V) = cache.put(key, value) - override def notifyRemove(key: K) = keyToRdds.remove(key) + override def notifyRemove(key: K) = cache.remove(key) override def notifyGet(key: K) = Unit - override def keysOfCachedEntries: Seq[K] = keyToRdds.keySet.toSeq + override def keysOfCachedEntries: Seq[K] = cache.keySet.toSeq override def hitRate = 1.0 @@ -42,6 +45,9 @@ class CacheAllPolicy[K, V] extends CachePolicy[K, V] { } +/** + * + */ trait CachePolicy[K, V] { protected var _loadFunc: (K => V) = _ @@ -89,10 +95,16 @@ object CachePolicy { def instantiateWithUserSpecs[K, V](str: String, fallbackMaxSize: Int): CachePolicy[K, V] = { val firstParenPos = str.indexOf('(') - val classStr = str.slice(0, firstParenPos) - val strArgs = str.substring(firstParenPos + 1, str.lastIndexOf(')')).split(',') - val policy = Class.forName(classStr).newInstance.asInstanceOf[CachePolicy[K, V]] - policy.initializeWithUserSpecs(strArgs, fallbackMaxSize) - return policy + if (firstParenPos == -1) { + val policy = Class.forName(str).newInstance.asInstanceOf[CachePolicy[K, V]] + policy.initializeWithUserSpecs(Array.empty[String], fallbackMaxSize) + return policy + } else { + val classStr = str.slice(0, firstParenPos) + val strArgs = str.substring(firstParenPos + 1, str.lastIndexOf(')')).split(',') + val policy = Class.forName(classStr).newInstance.asInstanceOf[CachePolicy[K, V]] + policy.initializeWithUserSpecs(strArgs, fallbackMaxSize) + return policy + } } } diff --git a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala index 4d1e17b5..553c1423 100644 --- a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala +++ b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala @@ -49,9 +49,12 @@ sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { extends LinkedHashMap[K, V](maxSize, 0.75F, accessOrder) { override def removeEldestEntry(eldest: Entry[K, V]): Boolean = { - _evictionFunc(eldest.getKey, eldest.getValue) - _evictionCount += 1 - return (size() > maxSize) + val shouldRemove = (size() > maxSize) + if (shouldRemove) { + _evictionFunc(eldest.getKey, eldest.getValue) + _evictionCount += 1 + } + return shouldRemove } } @@ -68,7 +71,13 @@ sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { override def notifyPut(key: K, value: V): Unit = { assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) - this.synchronized { _cache.put(key, value) } + this.synchronized { + val oldValue = _cache.put(key, value) + if (oldValue != null) { + _evictionFunc(key, oldValue) + _evictionCount += 1 + } + } } override def notifyRemove(key: K): Unit = { @@ -83,8 +92,8 @@ sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { _cache.get(key) _hitCount += 1L } else { - val retrievedValue = _loadFunc(key) - _cache.put(key, retrievedValue) + val loadedValue = _loadFunc(key) + _cache.put(key, loadedValue) _missCount += 1L } } @@ -100,7 +109,7 @@ sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { override def hitRate: Double = { this.synchronized { val requestCount = _missCount + _hitCount - val rate = if (requestCount == 0L) 1.0 else (_hitCount / requestCount) + val rate = if (requestCount == 0L) 1.0 else (_hitCount.toDouble / requestCount) return rate } } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index c50c7da7..91d3275f 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -115,8 +115,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { create table %s(key int, value string) partitioned by (keypart int) tblproperties('shark.cache' = 'true', - 'shark.cache.partition.cachePolicy.maxSize' = '%d', - 'shark.cache.partition.cachePolicy.class' = '%s', + 'shark.cache.policy.maxSize' = '%d', + 'shark.cache.policy' = '%s', 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') """.format( tableName, @@ -506,14 +506,25 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // RDD(partition) eviction policy for cached Hive-partititioned tables ////////////////////////////////////////////////////////////////////////////// - test("shark.memstore2.LRUCachePolicy is the default policy") { - val tableName = "lru_default_policy_cached" - sc.runSql("""create table lru_default_policy_cached(key int, value string) + test("shark.memstore2.CacheAllPolicy is the default policy") { + val tableName = "default_policy_cached" + sc.runSql("""create table default_policy_cached(key int, value string) partitioned by (keypart int)""") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get val cachePolicy = partitionedTable.cachePolicy - assert(cachePolicy.isInstanceOf[shark.memstore2.LRUCachePolicy[_, _]]) + assert(cachePolicy.isInstanceOf[shark.memstore2.CacheAllPolicy[_, _]]) + } + + test("LRU: RDDs are not evicted if the cache isn't full.") { + val tableName = "evict_partitions_maxSize" + val partitionedTable = createCachedPartitionedTable( + tableName, + 2 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) } test("LRU: RDDs are evicted when the max size is reached.") { @@ -623,7 +634,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { "shark.memstore2.LRUCachePolicy") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + val lvl = RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) + assert(lvl == StorageLevel.MEMORY_AND_DISK, "got: " + lvl) sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) select * from test""") assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) @@ -646,10 +658,10 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") val lruCachePolicy = partitionedTable.cachePolicy - val hitRate = lruCachePolicy.getHitRate - assert(hitRate == 1.0) - val evictionCount = lruCachePolicy.getEvictionCount - assert(evictionCount == 0) + val hitRate = lruCachePolicy.hitRate + assert(hitRate == 1.0, "got: " + hitRate) + val evictionCount = lruCachePolicy.evictionCount + assert(evictionCount == 0, "got: " + evictionCount) } ////////////////////////////////////////////////////////////////////////////// @@ -660,7 +672,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("drop table if exists adw") sc.sql("""create table adw TBLPROPERTIES ("shark.cache" = "true") as select cast(key as int) as k, val from test""") - expectSql("select count(k) from adw where val='val_487' group by 1 having count(1) > 0","1") + expectSql("select count(k) from adw where val='val_487' group by 1 having count(1) > 0", "1") } ////////////////////////////////////////////////////////////////////////////// From 6e992a47e6df7eeb15f4b3e7487ed6fa382f4d49 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 07:22:34 -0700 Subject: [PATCH 152/331] Flatten UnionRDDs created from multiple INSERT INTOs. --- .../shark/execution/MemoryStoreSinkOperator.scala | 7 ++++--- src/main/scala/shark/execution/RDDUtils.scala | 15 ++++++++++++++- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 2d8de473..a0c90bd0 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -156,7 +156,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { var queryOutputRDD = outputRDD if (useUnionRDD) { // Handle an INSERT INTO command. - var previousRDDOpt: Option[RDD[_]] = + var previousRDDOpt: Option[RDD[TablePartition]] = if (isHivePartitioned) { val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get partitionedTable.getPartition(hivePartitionKey) @@ -165,9 +165,10 @@ class MemoryStoreSinkOperator extends TerminalOperator { } outputRDD = previousRDDOpt match { case Some(previousRDD) => { - // If the table or a Hive-partition for the INSERT has already been created, take a - // union of the current data and the SELECT output. + // If the RDD for a table or Hive-partition has already been created, then take a union + // of the current data and the SELECT output. hasPreviousRDDForUnion = true + RDDUtils.unionAndFlatten(queryOutputRDD, previousRDD) queryOutputRDD.union(previousRDD.asInstanceOf[RDD[TablePartition]]) } // This is an INSERT into a new Hive-partition. diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 5a56443c..71250be6 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -34,6 +34,18 @@ import shark.SharkEnv */ object RDDUtils { + def unionAndFlatten[T: ClassManifest]( + rdd: RDD[T], + otherRdd: RDD[T]): RDD[T] = { + val unionedRdd = otherRdd match { + case unionRdd: UnionRDD[_] => { + new UnionRDD(rdd.context, (unionRdd.rdds :+ rdd)) + } + case _ => rdd.union(otherRdd) + } + return unionedRdd + } + def getStorageLevelOfRDD(rdd: RDD[_]): StorageLevel = { rdd match { case u: UnionRDD[_] => { @@ -59,7 +71,8 @@ object RDDUtils { rdds.foldLeft(StorageLevel.NONE) { (s, r) => { if (s == StorageLevel.NONE) { - // Mutally recursive if 'r' is a UnionRDD. + // Mutally recursive if `r` is a UnionRDD. However, this shouldn't happen in Shark, since + // UnionRDDs from successive INSERT INTOs are created through #unionAndFlatten(). getStorageLevelOfRDD(r) } else { // Some RDD in 'rdds' is persisted in memory or disk, so return early. From 199c9c1cdfb5c68a553c56c9630d66d2da67ecd4 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 07:27:12 -0700 Subject: [PATCH 153/331] Clarify what a "regular" create table command is. --- .../shark/parse/SharkSemanticAnalyzer.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 0bf0a8f0..dea392d9 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -351,6 +351,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // should have everything (e.g. isCTAS(), partCols). Note that the QB might not be // accessible from getParseContext(), since the SemanticAnalyzer#analyzeInternal() // doesn't set (this.qb = qb) for a non-CTAS. + // True if the command is a CREATE TABLE, but not a CTAS. var isRegularCreateTable = true var isHivePartitioned = false @@ -365,15 +366,14 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } var ddlTasks: Seq[DDLTask] = Nil - val createTableDesc = - if (isRegularCreateTable) { - // Unfortunately, we have to comb the root tasks because for CREATE TABLE, - // SemanticAnalyzer#analyzeCreateTable() does't set the CreateTableDesc in its QB. - ddlTasks = rootTasks.filter(_.isInstanceOf[DDLTask]).asInstanceOf[Seq[DDLTask]] - if (ddlTasks.isEmpty) null else ddlTasks.head.getWork.getCreateTblDesc - } else { - getParseContext.getQB.getTableDesc - } + val createTableDesc = if (isRegularCreateTable) { + // Unfortunately, we have to comb the root tasks because for CREATE TABLE, + // SemanticAnalyzer#analyzeCreateTable() does't set the CreateTableDesc in its QB. + ddlTasks = rootTasks.filter(_.isInstanceOf[DDLTask]).asInstanceOf[Seq[DDLTask]] + if (ddlTasks.isEmpty) null else ddlTasks.head.getWork.getCreateTblDesc + } else { + getParseContext.getQB.getTableDesc + } // 'createTableDesc' is NULL if there is an IF NOT EXISTS condition and the target table // already exists. From e9e3079413271bca925eae028b9186c9ab80fa0f Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 08:24:34 -0700 Subject: [PATCH 154/331] Minor style changes --- src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala | 6 +++--- src/test/scala/shark/SQLSuite.scala | 8 +++----- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala index 553c1423..8cfc7cc6 100644 --- a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala +++ b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala @@ -27,7 +27,7 @@ class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { super.initializeInternals(loadFunc, evictionFunc) - _cache = new LinkedMapCache(true /* accessOrder */) + _cache = new LinkedMapCache(true /* evictUsingAccessOrder */) } } @@ -45,8 +45,8 @@ class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { - class LinkedMapCache(accessOrder: Boolean = false) - extends LinkedHashMap[K, V](maxSize, 0.75F, accessOrder) { + class LinkedMapCache(evictUsingAccessOrder: Boolean = false) + extends LinkedHashMap[K, V](maxSize, 0.75F, evictUsingAccessOrder) { override def removeEldestEntry(eldest: Entry[K, V]): Boolean = { val shouldRemove = (size() > maxSize) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 91d3275f..7691db8c 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -650,7 +650,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) } - test("LRU: record cache stats") { + test("LRU: cache stats are recorded") { val tableName = "should_record_partition_cache_stats" val partitionedTable = createCachedPartitionedTable( tableName, @@ -658,10 +658,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") val lruCachePolicy = partitionedTable.cachePolicy - val hitRate = lruCachePolicy.hitRate - assert(hitRate == 1.0, "got: " + hitRate) - val evictionCount = lruCachePolicy.evictionCount - assert(evictionCount == 0, "got: " + evictionCount) + assert(lruCachePolicy.hitRate == 1.0) + assert(lruCachePolicy.evictionCount == 0) } ////////////////////////////////////////////////////////////////////////////// From 50507b5feadf1f43eb2cc60ce8ebae6716505322 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 08:33:02 -0700 Subject: [PATCH 155/331] New test suite for CachePolicy. --- src/test/scala/shark/SQLSuite.scala | 12 -- .../shark/memstore2/CachePolicySuite.scala | 137 ++++++++++++++++++ 2 files changed, 137 insertions(+), 12 deletions(-) create mode 100644 src/test/scala/shark/memstore2/CachePolicySuite.scala diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 7691db8c..0a4e4f55 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -650,18 +650,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) } - test("LRU: cache stats are recorded") { - val tableName = "should_record_partition_cache_stats" - val partitionedTable = createCachedPartitionedTable( - tableName, - 1 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - val lruCachePolicy = partitionedTable.cachePolicy - assert(lruCachePolicy.hitRate == 1.0) - assert(lruCachePolicy.evictionCount == 0) - } - ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// diff --git a/src/test/scala/shark/memstore2/CachePolicySuite.scala b/src/test/scala/shark/memstore2/CachePolicySuite.scala new file mode 100644 index 00000000..90f3a0f8 --- /dev/null +++ b/src/test/scala/shark/memstore2/CachePolicySuite.scala @@ -0,0 +1,137 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import org.scalatest.FunSuite + +import scala.collection.mutable.HashMap + +class CachePolicySuite extends FunSuite { + + case class TestValue(var value: Int, var isCached: Boolean) + + class IdentifyKVGen(max: Int) { + val kvMap = new HashMap[Int, TestValue]() + for (i <- 0 until max) { + kvMap(i) = TestValue(i, isCached = false) + } + + def loadFunc(key: Int) = { + val value = kvMap(key) + value.isCached = true + value + } + + def evictionFunc(key: Int, value: TestValue) = { + value.isCached = false + } + } + + test("LRU policy") { + val kvGen = new IdentifyKVGen(20) + val loadFunc = kvGen.loadFunc _ + val evictFunc = kvGen.evictionFunc _ + val cacheSize = 10 + val lru = new LRUCachePolicy[Int, TestValue]() + lru.initializeWithUserSpecs(Array.empty[String], cacheSize) + lru.initializeInternals(loadFunc, evictFunc) + + // Load KVs 0-9. + (0 to 9).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))) + + // Reorder access order by getting keys 2-4. + (2 to 4).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(0, 1, 5, 6, 7, 8, 9, 2, 3, 4))) + + // Get keys 10-12, which should evict (0, 1, 5). + (10 to 12).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(6, 7, 8, 9, 2, 3, 4, 10, 11, 12))) + // Make sure the eviction function ran. + assert(!kvGen.kvMap(0).isCached) + assert(!kvGen.kvMap(1).isCached) + assert(!kvGen.kvMap(5).isCached) + + // Reorder access order by getting keys (6, 8, 2). + lru.notifyGet(6); lru.notifyGet(8); lru.notifyGet(2) + assert(lru.keysOfCachedEntries.equals(Seq(7, 9, 3, 4, 10, 11, 12, 6, 8, 2))) + + // Remove 9, 4 and add 13, 14, 15. 7 should be evicted. + lru.notifyRemove(9); lru.notifyRemove(4) + (13 to 15).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(3, 10, 11, 12, 6, 8, 2, 13, 14, 15))) + assert(!kvGen.kvMap(7).isCached) + } + + test("FIFO policy") { + val kvGen = new IdentifyKVGen(15) + val loadFunc = kvGen.loadFunc _ + val evictFunc = kvGen.evictionFunc _ + val cacheSize = 5 + val fifo = new FIFOCachePolicy[Int, TestValue]() + fifo.initializeWithUserSpecs(Array.empty[String], cacheSize) + fifo.initializeInternals(loadFunc, evictFunc) + + // Load KVs 0-4. + (0 to 4).map(fifo.notifyGet(_)) + assert(fifo.keysOfCachedEntries.equals(Seq(0, 1, 2, 3, 4))) + + // Get 0-8, which should evict 0-3. + (0 to 8).map(fifo.notifyGet(_)) + assert(fifo.keysOfCachedEntries.equals(Seq(4, 5, 6, 7, 8))) + + // Remove 4, 6 and add 9-12. 5 and 7 should be evicted. + fifo.notifyRemove(4); fifo.notifyRemove(6) + (9 to 12).map(fifo.notifyGet(_)) + assert(fifo.keysOfCachedEntries.equals(Seq(8, 9, 10, 11, 12))) + } + + test("Policy classes instantiated from a string, with maxSize argument") { + val lruStr = "shark.memstore2.LRUCachePolicy(5)" + val lru = CachePolicy.instantiateWithUserSpecs(lruStr, 10) + assert(lru.maxSize == 5) + val fifoStr = "shark.memstore2.FIFOCachePolicy(5)" + val fifo = CachePolicy.instantiateWithUserSpecs(fifoStr, 10) + assert(fifo.maxSize == 5) + } + + test("Cache stats are recorded") { + val kvGen = new IdentifyKVGen(20) + val loadFunc = kvGen.loadFunc _ + val evictFunc = kvGen.evictionFunc _ + val cacheSize = 5 + val lru = new LRUCachePolicy[Int, TestValue]() + lru.initializeWithUserSpecs(Array.empty[String], cacheSize) + lru.initializeInternals(loadFunc, evictFunc) + + // Hit rate should start at 1.0 + assert(lru.hitRate == 1.0) + + (0 to 4).map(lru.notifyGet(_)) + assert(lru.hitRate == 0.0) + + // Get 1, 2, 3, which should bring the hit rate to 0.375. + (1 to 3).map(lru.notifyGet(_)) + assert(lru.hitRate == 0.375) + + // Get 2-5, which brings the hit rate up to 0.50. + (2 to 5).map(lru.notifyGet(_)) + assert(lru.evictionCount == 1) + assert(lru.hitRate == 0.50) + } +} \ No newline at end of file From c9c271c9af2aa1977a9349b7d20a188c69984031 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Wed, 23 Oct 2013 12:10:54 -0700 Subject: [PATCH 156/331] enable tachyon using 0.3.0 release --- project/SharkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index c9f32bb9..b014473f 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -44,7 +44,7 @@ object SharkBuild extends Build { val YARN_ENABLED = env("SHARK_YARN").getOrElse("false").toBoolean // Whether to build Shark with Tachyon jar. - val TACHYON_ENABLED = false + val TACHYON_ENABLED = true lazy val root = Project( id = "root", @@ -128,7 +128,7 @@ object SharkBuild extends Build { "net.java.dev.jets3t" % "jets3t" % "0.7.1", "com.novocode" % "junit-interface" % "0.8" % "test") ++ (if (YARN_ENABLED) Some("org.apache.spark" %% "spark-yarn" % SPARK_VERSION) else None).toSeq ++ - (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0-SNAPSHOT" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm)) else None).toSeq + (if (TACHYON_ENABLED) Some("org.tachyonproject" % "tachyon" % "0.3.0" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm)) else None).toSeq ) ++ org.scalastyle.sbt.ScalastylePlugin.Settings def assemblyProjSettings = Seq( From 4e72dc8c0b5585b8864f3498aa778544262b304c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 23 Oct 2013 19:23:39 -0700 Subject: [PATCH 157/331] Remove debug output accidentally committed in c4b6d50d8c41866dc0a43bdf708c2dba8964a0d0 --- src/main/scala/shark/execution/ScriptOperator.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 69eb1c09..1881c8a3 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -256,7 +256,6 @@ object ScriptOperator { with LogHelper { override def uncaughtException(thread: Thread, throwable: Throwable) { - Thread.sleep(7000) throwable match { case ioe: IOException => { // Check whether the IOException should be re-thrown by the parent thread. @@ -269,7 +268,6 @@ object ScriptOperator { .format(HiveConf.ConfVars.ALLOWPARTIALCONSUMP.toString)) throw ioe } - println("thread finished...") context.synchronized { context.addOnCompleteCallback(onCompleteCallback) } From da3ecf224084a8dd5d8445377f40358b2d00b058 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 25 Oct 2013 12:42:18 -0700 Subject: [PATCH 158/331] Moved all cache eviction policies to CachePolicy.scala --- .../scala/shark/memstore2/CachePolicy.scala | 145 +++++++++++++++--- .../memstore2/LinkedMapBasedPolicy.scala | 119 -------------- 2 files changed, 123 insertions(+), 141 deletions(-) delete mode 100644 src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index e8bdf48b..a7b82a9c 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -18,35 +18,18 @@ package shark.memstore2 import java.util.concurrent.ConcurrentHashMap +import java.util.LinkedHashMap +import java.util.Map.Entry import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD -/** - * A cache that never evicts entries. - */ -class CacheAllPolicy[K, V] extends CachePolicy[K, V] { - - // Track the entries in the cache, so that keysOfCachedEntries() returns a valid result. - var cache = new ConcurrentHashMap[K, V]() - - override def notifyPut(key: K, value: V) = cache.put(key, value) - - override def notifyRemove(key: K) = cache.remove(key) - - override def notifyGet(key: K) = Unit - - override def keysOfCachedEntries: Seq[K] = cache.keySet.toSeq - - override def hitRate = 1.0 - - override def evictionCount = 0L -} - /** - * + * An general interface for pluggable cache eviction policies in Shark. + * One example of usage is to control persistance levels of RDDs that represent a table's + * Hive-partitions. */ trait CachePolicy[K, V] { @@ -108,3 +91,121 @@ object CachePolicy { } } } + + +/** + * A cache that never evicts entries. + */ +class CacheAllPolicy[K, V] extends CachePolicy[K, V] { + + // Track the entries in the cache, so that keysOfCachedEntries() returns a valid result. + var cache = new ConcurrentHashMap[K, V]() + + override def notifyPut(key: K, value: V) = cache.put(key, value) + + override def notifyRemove(key: K) = cache.remove(key) + + override def notifyGet(key: K) = Unit + + override def keysOfCachedEntries: Seq[K] = cache.keySet.toSeq + + override def hitRate = 1.0 + + override def evictionCount = 0L +} + + +class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { + + override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { + super.initializeInternals(loadFunc, evictionFunc) + _cache = new LinkedMapCache(true /* evictUsingAccessOrder */) + } + +} + + +class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { + + override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { + super.initializeInternals(loadFunc, evictionFunc) + _cache = new LinkedMapCache() + } + +} + + +sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { + + class LinkedMapCache(evictUsingAccessOrder: Boolean = false) + extends LinkedHashMap[K, V](maxSize, 0.75F, evictUsingAccessOrder) { + + override def removeEldestEntry(eldest: Entry[K, V]): Boolean = { + val shouldRemove = (size() > maxSize) + if (shouldRemove) { + _evictionFunc(eldest.getKey, eldest.getValue) + _evictionCount += 1 + } + return shouldRemove + } + } + + protected var _cache: LinkedMapCache = _ + protected var _isInitialized = false + protected var _hitCount: Long = 0L + protected var _missCount: Long = 0L + protected var _evictionCount: Long = 0L + + override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { + super.initializeInternals(loadFunc, evictionFunc) + _isInitialized = true + } + + override def notifyPut(key: K, value: V): Unit = { + assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { + val oldValue = _cache.put(key, value) + if (oldValue != null) { + _evictionFunc(key, oldValue) + _evictionCount += 1 + } + } + } + + override def notifyRemove(key: K): Unit = { + assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { _cache.remove(key) } + } + + override def notifyGet(key: K): Unit = { + assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) + this.synchronized { + if (_cache.contains(key)) { + _cache.get(key) + _hitCount += 1L + } else { + val loadedValue = _loadFunc(key) + _cache.put(key, loadedValue) + _missCount += 1L + } + } + } + + override def keysOfCachedEntries: Seq[K] = { + assert(_isInitialized, "Must initialize() LRUCachePolicy.") + this.synchronized { + return _cache.keySet.toSeq + } + } + + override def hitRate: Double = { + this.synchronized { + val requestCount = _missCount + _hitCount + val rate = if (requestCount == 0L) 1.0 else (_hitCount.toDouble / requestCount) + return rate + } + } + + override def evictionCount = _evictionCount + +} diff --git a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala b/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala deleted file mode 100644 index 8cfc7cc6..00000000 --- a/src/main/scala/shark/memstore2/LinkedMapBasedPolicy.scala +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import java.util.LinkedHashMap -import java.util.Map.Entry - -import scala.collection.JavaConversions._ - - -class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { - - override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { - super.initializeInternals(loadFunc, evictionFunc) - _cache = new LinkedMapCache(true /* evictUsingAccessOrder */) - } - -} - - -class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { - - override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { - super.initializeInternals(loadFunc, evictionFunc) - _cache = new LinkedMapCache() - } - -} - - -sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { - - class LinkedMapCache(evictUsingAccessOrder: Boolean = false) - extends LinkedHashMap[K, V](maxSize, 0.75F, evictUsingAccessOrder) { - - override def removeEldestEntry(eldest: Entry[K, V]): Boolean = { - val shouldRemove = (size() > maxSize) - if (shouldRemove) { - _evictionFunc(eldest.getKey, eldest.getValue) - _evictionCount += 1 - } - return shouldRemove - } - } - - protected var _cache: LinkedMapCache = _ - protected var _isInitialized = false - protected var _hitCount: Long = 0L - protected var _missCount: Long = 0L - protected var _evictionCount: Long = 0L - - override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { - super.initializeInternals(loadFunc, evictionFunc) - _isInitialized = true - } - - override def notifyPut(key: K, value: V): Unit = { - assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) - this.synchronized { - val oldValue = _cache.put(key, value) - if (oldValue != null) { - _evictionFunc(key, oldValue) - _evictionCount += 1 - } - } - } - - override def notifyRemove(key: K): Unit = { - assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) - this.synchronized { _cache.remove(key) } - } - - override def notifyGet(key: K): Unit = { - assert(_isInitialized, "Must initialize() %s.".format(this.getClass.getName)) - this.synchronized { - if (_cache.contains(key)) { - _cache.get(key) - _hitCount += 1L - } else { - val loadedValue = _loadFunc(key) - _cache.put(key, loadedValue) - _missCount += 1L - } - } - } - - override def keysOfCachedEntries: Seq[K] = { - assert(_isInitialized, "Must initialize() LRUCachePolicy.") - this.synchronized { - return _cache.keySet.toSeq - } - } - - override def hitRate: Double = { - this.synchronized { - val requestCount = _missCount + _hitCount - val rate = if (requestCount == 0L) 1.0 else (_hitCount.toDouble / requestCount) - return rate - } - } - - override def evictionCount = _evictionCount - -} From 736fb540f9c5779a72ef0ceed9183e6377e48a05 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 25 Oct 2013 12:43:17 -0700 Subject: [PATCH 159/331] Moved getStorageLevelOfRDD() to a TestUtils class, since it's no longer used in non-test code. --- src/main/scala/shark/execution/RDDUtils.scala | 37 ---------------- .../memstore2/PartitionedMemoryTable.scala | 8 +--- src/test/scala/shark/SQLSuite.scala | 43 +++++++++---------- src/test/scala/shark/TestUtils.scala | 39 +++++++++++++++++ 4 files changed, 61 insertions(+), 66 deletions(-) diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 71250be6..7d18dd53 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -23,7 +23,6 @@ import com.google.common.collect.{Ordering => GOrdering} import org.apache.spark.{HashPartitioner, Partitioner, RangePartitioner} import org.apache.spark.rdd.{RDD, ShuffledRDD, UnionRDD} -import org.apache.spark.storage.StorageLevel import shark.SharkEnv @@ -46,42 +45,6 @@ object RDDUtils { return unionedRdd } - def getStorageLevelOfRDD(rdd: RDD[_]): StorageLevel = { - rdd match { - case u: UnionRDD[_] => { - // Find the storage level of a UnionRDD from the storage levels of RDDs that compose it. - // A StorageLevel.NONE is returned if all of those RDDs have StorageLevel.NONE. - // Mutually recursive if any RDD in 'u.rdds' is a UnionRDD. - getStorageLevelOfRDDs(u.rdds) - } - case _ => rdd.getStorageLevel - } - } - - /** - * Returns the storage level of a sequence of RDDs, interpreted as the storage level of the first - * RDD in the sequence that is persisted in memory or on disk. This works because for Shark's use - * case, all RDDs for a non-partitioned table should have the same storage level. An RDD for a - * partitioned table could be StorageLevel.NONE if it was unpersisted by the partition eviction - * policy. - * - * @param rdds The sequence of RDDs to find the StorageLevel of. - */ - def getStorageLevelOfRDDs(rdds: Seq[RDD[_]]): StorageLevel = { - rdds.foldLeft(StorageLevel.NONE) { - (s, r) => { - if (s == StorageLevel.NONE) { - // Mutally recursive if `r` is a UnionRDD. However, this shouldn't happen in Shark, since - // UnionRDDs from successive INSERT INTOs are created through #unionAndFlatten(). - getStorageLevelOfRDD(r) - } else { - // Some RDD in 'rdds' is persisted in memory or disk, so return early. - return s - } - } - } - } - def unpersistRDD(rdd: RDD[_]): RDD[_] = { rdd match { case u: UnionRDD[_] => { diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 6d72269a..42e39193 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -114,13 +114,7 @@ class PartitionedMemoryTable( val loadFunc: String => RDDValue = (partitionKey: String) => { val rddValue = _keyToPartitions.get(partitionKey).get - // RDDUtils#getStorageLevelOfRDD() will return StorageLevel.NONE only if all parent RDDs of - // a UnionRDD are not persisted. That function is used to check the RDD's storage level in - // order to avoid directly persisting an UnionRDD comprising RDDs already persisted in - // memory, which leads to duplicate data. - if (RDDUtils.getStorageLevelOfRDD(rddValue.rdd) == StorageLevel.NONE) { - rddValue.rdd.persist(preferredStorageLevel) - } + rddValue.rdd.persist(preferredStorageLevel) rddValue } // The evictionFunc will unpersist the RDD. diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 0a4e4f55..1d8ffcc0 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.PartitionedMemoryTable -import shark.execution.RDDUtils class SQLSuite extends FunSuite with BeforeAndAfterAll { @@ -524,7 +523,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) } test("LRU: RDDs are evicted when the max size is reached.") { @@ -535,10 +534,10 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("""insert into table evict_partitions_maxSize partition(keypart = 4) select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) } test("LRU: RDD eviction accounts for partition scans - a cache.get()") { @@ -550,14 +549,14 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { "shark.memstore2.LRUCachePolicy") val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("select count(1) from evict_partitions_with_get where keypart = 1") sc.runSql("""insert into table evict_partitions_with_get partition(keypart = 4) select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) } test("LRU: RDD eviction accounts for INSERT INTO - a cache.get().") { @@ -570,17 +569,17 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 1) select * from test""") sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 4) select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) assert(keypart2StorageLevel == StorageLevel.NONE) } @@ -594,17 +593,17 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) sc.runSql("""insert overwrite table evict_partitions_insert_overwrite partition(keypart = 1) select * from test""") sc.runSql("""insert into table evict_partitions_insert_overwrite partition(keypart = 4) select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(RDDUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) assert(keypart2StorageLevel == StorageLevel.NONE) } @@ -622,7 +621,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""insert into table evict_partitions_removals partition(keypart = 5) select * from test""") val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) } test("LRU: get() reloads an RDD previously unpersist()'d.") { @@ -634,18 +633,18 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { "shark.memstore2.LRUCachePolicy") assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val lvl = RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) + val lvl = TestUtils.getStorageLevelOfRDD(keypart1RDD.get) assert(lvl == StorageLevel.MEMORY_AND_DISK, "got: " + lvl) sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) select * from test""") - assert(RDDUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and // cause eviction of the RDD for partition (keypart = 2). sc.runSql("select count(1) from reload_evicted_partition where keypart = 1") assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - val keypart2StorageLevel = RDDUtils.getStorageLevelOfRDD(keypart2RDD.get) + val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) assert(keypart2StorageLevel == StorageLevel.NONE, "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) } diff --git a/src/test/scala/shark/TestUtils.scala b/src/test/scala/shark/TestUtils.scala index df2c264a..8bf0fd6f 100644 --- a/src/test/scala/shark/TestUtils.scala +++ b/src/test/scala/shark/TestUtils.scala @@ -24,6 +24,9 @@ import java.util.{Date, HashMap => JHashMap} import org.apache.hadoop.hive.common.LogUtils import org.apache.hadoop.hive.common.LogUtils.LogInitializationException +import org.apache.spark.rdd.{RDD, UnionRDD} +import org.apache.spark.storage.StorageLevel + object TestUtils { @@ -48,6 +51,42 @@ object TestUtils { } } + def getStorageLevelOfRDD(rdd: RDD[_]): StorageLevel = { + rdd match { + case u: UnionRDD[_] => { + // Find the storage level of a UnionRDD from the storage levels of RDDs that compose it. + // A StorageLevel.NONE is returned if all of those RDDs have StorageLevel.NONE. + // Mutually recursive if any RDD in 'u.rdds' is a UnionRDD. + getStorageLevelOfRDDs(u.rdds) + } + case _ => rdd.getStorageLevel + } + } + + /** + * Returns the storage level of a sequence of RDDs, interpreted as the storage level of the first + * RDD in the sequence that is persisted in memory or on disk. This works because for Shark's use + * case, all RDDs for a non-partitioned table should have the same storage level. An RDD for a + * partitioned table could be StorageLevel.NONE if it was unpersisted by the partition eviction + * policy. + * + * @param rdds The sequence of RDDs to find the StorageLevel of. + */ + def getStorageLevelOfRDDs(rdds: Seq[RDD[_]]): StorageLevel = { + rdds.foldLeft(StorageLevel.NONE) { + (s, r) => { + if (s == StorageLevel.NONE) { + // Mutally recursive if `r` is a UnionRDD. However, this shouldn't happen in Shark, since + // UnionRDDs from successive INSERT INTOs are created through #unionAndFlatten(). + getStorageLevelOfRDD(r) + } else { + // Some RDD in 'rdds' is persisted in memory or disk, so return early. + return s + } + } + } + } + // Don't use default arguments in the above functions because otherwise the JavaAPISuite // can't call those functions with default arguments. def getWarehousePath(): String = getWarehousePath("sql") From 1152dab5062a066affc546c9232ff0726d0bb335 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 25 Oct 2013 14:16:59 -0700 Subject: [PATCH 160/331] Make one initialize() call for CachePolicy. --- .../scala/shark/memstore2/CachePolicy.scala | 54 ++++++++++++------- .../memstore2/PartitionedMemoryTable.scala | 13 ++--- .../shark/memstore2/CachePolicySuite.scala | 22 +++----- 3 files changed, 49 insertions(+), 40 deletions(-) diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index a7b82a9c..a5b2cfcd 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -39,20 +39,22 @@ trait CachePolicy[K, V] { protected var _maxSize: Int = -1 - def initializeWithUserSpecs(args: Array[String], fallbackMaxSize: Int) { + def initialize( + strArgs: Array[String], + fallbackMaxSize: Int, + loadFunc: K => V, + evictionFunc: (K, V) => Unit) { + _loadFunc = loadFunc + _evictionFunc = evictionFunc + // By default, only initialize the `maxSize` from user specifications. - args.size match { + strArgs.size match { case 0 => _maxSize = fallbackMaxSize - case 1 => _maxSize = args.head.toInt + case 1 => _maxSize = strArgs.head.toInt case _ => throw new Exception("Accpted format: %s(maxSize: Int)".format(this.getClass.getName)) } - } - - def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { - require(maxSize > 0, "Must specify a maxSize before initializing CachePolicy internals.") - _loadFunc = loadFunc - _evictionFunc = evictionFunc + require(maxSize > 0, "Size given to cache eviction policy must be > 1") } def notifyPut(key: K, value: V): Unit @@ -76,17 +78,21 @@ trait CachePolicy[K, V] { object CachePolicy { - def instantiateWithUserSpecs[K, V](str: String, fallbackMaxSize: Int): CachePolicy[K, V] = { + def instantiateWithUserSpecs[K, V]( + str: String, + fallbackMaxSize: Int, + loadFunc: K => V, + evictionFunc: (K, V) => Unit): CachePolicy[K, V] = { val firstParenPos = str.indexOf('(') if (firstParenPos == -1) { val policy = Class.forName(str).newInstance.asInstanceOf[CachePolicy[K, V]] - policy.initializeWithUserSpecs(Array.empty[String], fallbackMaxSize) + policy.initialize(Array.empty[String], fallbackMaxSize, loadFunc, evictionFunc) return policy } else { val classStr = str.slice(0, firstParenPos) val strArgs = str.substring(firstParenPos + 1, str.lastIndexOf(')')).split(',') val policy = Class.forName(classStr).newInstance.asInstanceOf[CachePolicy[K, V]] - policy.initializeWithUserSpecs(strArgs, fallbackMaxSize) + policy.initialize(strArgs, fallbackMaxSize, loadFunc, evictionFunc) return policy } } @@ -117,8 +123,12 @@ class CacheAllPolicy[K, V] extends CachePolicy[K, V] { class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { - override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { - super.initializeInternals(loadFunc, evictionFunc) + override def initialize( + strArgs: Array[String], + fallbackMaxSize: Int, + loadFunc: K => V, + evictionFunc: (K, V) => Unit) { + super.initialize(strArgs, fallbackMaxSize, loadFunc, evictionFunc) _cache = new LinkedMapCache(true /* evictUsingAccessOrder */) } @@ -127,8 +137,12 @@ class LRUCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { class FIFOCachePolicy[K, V] extends LinkedMapBasedPolicy[K, V] { - override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { - super.initializeInternals(loadFunc, evictionFunc) + override def initialize( + strArgs: Array[String], + fallbackMaxSize: Int, + loadFunc: K => V, + evictionFunc: (K, V) => Unit) { + super.initialize(strArgs, fallbackMaxSize, loadFunc, evictionFunc) _cache = new LinkedMapCache() } @@ -156,8 +170,12 @@ sealed abstract class LinkedMapBasedPolicy[K, V] extends CachePolicy[K, V] { protected var _missCount: Long = 0L protected var _evictionCount: Long = 0L - override def initializeInternals(loadFunc: (K => V), evictionFunc: (K, V) => Unit) { - super.initializeInternals(loadFunc, evictionFunc) + override def initialize( + strArgs: Array[String], + fallbackMaxSize: Int, + loadFunc: K => V, + evictionFunc: (K, V) => Unit) { + super.initialize(strArgs, fallbackMaxSize, loadFunc, evictionFunc) _isInitialized = true } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 42e39193..fb63596d 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -84,8 +84,8 @@ class PartitionedMemoryTable( } def updatePartition( - partitionKey: String, - updatedRDD: RDD[TablePartition]): Option[RDD[TablePartition]] = { + partitionKey: String, + updatedRDD: RDD[TablePartition]): Option[RDD[TablePartition]] = { val rddValueOpt = _keyToPartitions.get(partitionKey) var prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) if (rddValueOpt.isDefined) { @@ -105,11 +105,7 @@ class PartitionedMemoryTable( return rddRemoved.map(_.rdd) } - def setPartitionCachePolicy( - cachePolicyStr: String, - fallbackMaxSize: Int) { - val newPolicy = CachePolicy.instantiateWithUserSpecs[String, RDDValue]( - cachePolicyStr, fallbackMaxSize) + def setPartitionCachePolicy(cachePolicyStr: String, fallbackMaxSize: Int) { // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. val loadFunc: String => RDDValue = (partitionKey: String) => { @@ -120,7 +116,8 @@ class PartitionedMemoryTable( // The evictionFunc will unpersist the RDD. val evictionFunc: (String, RDDValue) => Unit = (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) - newPolicy.initializeInternals(loadFunc, evictionFunc) + val newPolicy = CachePolicy.instantiateWithUserSpecs[String, RDDValue]( + cachePolicyStr, fallbackMaxSize, loadFunc, evictionFunc) _cachePolicy = newPolicy } diff --git a/src/test/scala/shark/memstore2/CachePolicySuite.scala b/src/test/scala/shark/memstore2/CachePolicySuite.scala index 90f3a0f8..9fe41d4a 100644 --- a/src/test/scala/shark/memstore2/CachePolicySuite.scala +++ b/src/test/scala/shark/memstore2/CachePolicySuite.scala @@ -44,12 +44,9 @@ class CachePolicySuite extends FunSuite { test("LRU policy") { val kvGen = new IdentifyKVGen(20) - val loadFunc = kvGen.loadFunc _ - val evictFunc = kvGen.evictionFunc _ val cacheSize = 10 val lru = new LRUCachePolicy[Int, TestValue]() - lru.initializeWithUserSpecs(Array.empty[String], cacheSize) - lru.initializeInternals(loadFunc, evictFunc) + lru.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) // Load KVs 0-9. (0 to 9).map(lru.notifyGet(_)) @@ -80,12 +77,9 @@ class CachePolicySuite extends FunSuite { test("FIFO policy") { val kvGen = new IdentifyKVGen(15) - val loadFunc = kvGen.loadFunc _ - val evictFunc = kvGen.evictionFunc _ val cacheSize = 5 val fifo = new FIFOCachePolicy[Int, TestValue]() - fifo.initializeWithUserSpecs(Array.empty[String], cacheSize) - fifo.initializeInternals(loadFunc, evictFunc) + fifo.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) // Load KVs 0-4. (0 to 4).map(fifo.notifyGet(_)) @@ -102,22 +96,22 @@ class CachePolicySuite extends FunSuite { } test("Policy classes instantiated from a string, with maxSize argument") { + val kvGen = new IdentifyKVGen(15) val lruStr = "shark.memstore2.LRUCachePolicy(5)" - val lru = CachePolicy.instantiateWithUserSpecs(lruStr, 10) + val lru = CachePolicy.instantiateWithUserSpecs( + lruStr, fallbackMaxSize = 10, kvGen.loadFunc _, kvGen.evictionFunc _) assert(lru.maxSize == 5) val fifoStr = "shark.memstore2.FIFOCachePolicy(5)" - val fifo = CachePolicy.instantiateWithUserSpecs(fifoStr, 10) + val fifo = CachePolicy.instantiateWithUserSpecs( + fifoStr, fallbackMaxSize = 10, kvGen.loadFunc _, kvGen.evictionFunc _) assert(fifo.maxSize == 5) } test("Cache stats are recorded") { val kvGen = new IdentifyKVGen(20) - val loadFunc = kvGen.loadFunc _ - val evictFunc = kvGen.evictionFunc _ val cacheSize = 5 val lru = new LRUCachePolicy[Int, TestValue]() - lru.initializeWithUserSpecs(Array.empty[String], cacheSize) - lru.initializeInternals(loadFunc, evictFunc) + lru.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) // Hit rate should start at 1.0 assert(lru.hitRate == 1.0) From f148b7729f01414ab9bef578fed5aa1bb0bff0fc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 Aug 2013 22:26:43 -0700 Subject: [PATCH 161/331] Added the initial functionality to convert RDDs into Shark in-memory tables. --- src/main/scala/shark/api/DataTypes.java | 81 ++++++++++-- src/main/scala/shark/api/RDDTable.scala | 115 ++++++++++++++++++ .../memstore2/TablePartitionBuilder.scala | 23 ++-- src/main/scala/shark/util/HiveUtils.scala | 77 ++++++++++++ 4 files changed, 273 insertions(+), 23 deletions(-) create mode 100644 src/main/scala/shark/api/RDDTable.scala create mode 100644 src/main/scala/shark/util/HiveUtils.scala diff --git a/src/main/scala/shark/api/DataTypes.java b/src/main/scala/shark/api/DataTypes.java index f8994c7f..1f71eb05 100644 --- a/src/main/scala/shark/api/DataTypes.java +++ b/src/main/scala/shark/api/DataTypes.java @@ -17,8 +17,13 @@ package shark.api; +import java.util.Date; import java.util.HashMap; import java.util.Map; +import java.sql.Timestamp; + +import scala.reflect.ClassManifest; +import scala.reflect.ClassManifest$; import org.apache.hadoop.hive.serde.Constants; @@ -30,19 +35,38 @@ public class DataTypes { // This list of types are defined in a Java class for better interoperability with Shark's // Java APIs. // Primitive types: - public static final DataType BOOLEAN = new DataType("boolean", Constants.BOOLEAN_TYPE_NAME, true); - public static final DataType TINYINT = new DataType("tinyint", Constants.TINYINT_TYPE_NAME, true); - public static final DataType SMALLINT = - new DataType("smallint", Constants.SMALLINT_TYPE_NAME, true); - public static final DataType INT = new DataType("int", Constants.INT_TYPE_NAME, true); - public static final DataType BIGINT = new DataType("bigint", Constants.BIGINT_TYPE_NAME, true); - public static final DataType FLOAT = new DataType("float", Constants.FLOAT_TYPE_NAME, true); - public static final DataType DOUBLE = new DataType("double", Constants.DOUBLE_TYPE_NAME, true); - public static final DataType STRING = new DataType("string", Constants.STRING_TYPE_NAME, true); - public static final DataType TIMESTAMP = - new DataType("timestamp", Constants.TIMESTAMP_TYPE_NAME, true); - public static final DataType DATE = new DataType("date", Constants.DATE_TYPE_NAME, true); - public static final DataType BINARY = new DataType("binary", Constants.BINARY_TYPE_NAME, true); + public static final DataType BOOLEAN = new DataType( + "boolean", Constants.BOOLEAN_TYPE_NAME, true); + + public static final DataType TINYINT = new DataType( + "tinyint", Constants.TINYINT_TYPE_NAME, true); + + public static final DataType SMALLINT = new DataType( + "smallint", Constants.SMALLINT_TYPE_NAME, true); + + public static final DataType INT = new DataType( + "int", Constants.INT_TYPE_NAME, true); + + public static final DataType BIGINT = new DataType( + "bigint", Constants.BIGINT_TYPE_NAME, true); + + public static final DataType FLOAT = new DataType( + "float", Constants.FLOAT_TYPE_NAME, true); + + public static final DataType DOUBLE = new DataType( + "double", Constants.DOUBLE_TYPE_NAME, true); + + public static final DataType STRING = new DataType( + "string", Constants.STRING_TYPE_NAME, true); + + public static final DataType TIMESTAMP = new DataType( + "timestamp", Constants.TIMESTAMP_TYPE_NAME, true); + + public static final DataType DATE = new DataType( + "date", Constants.DATE_TYPE_NAME, true); + + public static final DataType BINARY = new DataType( + "binary", Constants.BINARY_TYPE_NAME, true); // Complex types: // TODO: handle complex types. @@ -82,4 +106,35 @@ public static DataType fromHiveType(String hiveType) throws UnknownDataTypeExcep return type; } } + + public static DataType fromManifest(ClassManifest m) throws UnknownDataTypeException { + if (m.equals(m(Boolean.class)) || m.equals(ClassManifest$.MODULE$.Boolean())) { + return INT; + } else if (m.equals(m(Byte.class)) || m.equals(ClassManifest$.MODULE$.Byte())) { + return TINYINT; + } else if (m.equals(m(Short.class)) || m.equals(ClassManifest$.MODULE$.Short())) { + return SMALLINT; + } else if (m.equals(m(Integer.class)) || m.equals(ClassManifest$.MODULE$.Int())) { + return INT; + } else if (m.equals(m(Long.class)) || m.equals(ClassManifest$.MODULE$.Long())) { + return BIGINT; + } else if (m.equals(m(Float.class)) || m.equals(ClassManifest$.MODULE$.Float())) { + return FLOAT; + } else if (m.equals(m(Double.class)) || m.equals(ClassManifest$.MODULE$.Double())) { + return DOUBLE; + } else if (m.equals(m(String.class))) { + return STRING; + } else if (m.equals(m(Timestamp.class))) { + return TIMESTAMP; + } else if (m.equals(m(Date.class))) { + return DATE; + } else { + throw new UnknownDataTypeException(m.toString()); + } + // TODO: binary data type. + } + + private static ClassManifest m(Class cls) { + return ClassManifest$.MODULE$.fromClass(cls); + } } diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala new file mode 100644 index 00000000..3cb3e988 --- /dev/null +++ b/src/main/scala/shark/api/RDDTable.scala @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +import scala.collection.mutable.ArrayBuffer + +import shark.SharkEnv +import shark.memstore2.{TablePartitionStats, TablePartition, TablePartitionBuilder} +import shark.util.HiveUtils +import spark.RDD + + +class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { + + def saveAsTable(tableName: String, fields: Seq[String]) { + // Get a local copy of the manifests so we don't need to serialize this object. + val manifests = this.manifests + + val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) + + // Create the RDD object. + val rdd = self.mapPartitionsWithIndex { case(partitionIndex, iter) => + val ois = manifests.map(HiveUtils.getJavaPrimitiveObjectInspector) + val builder = new TablePartitionBuilder(ois, 1000000, shouldCompress = false) + + for (p <- iter) { + builder.incrementRowCount() + // TODO: this is not the most efficient code to do the insertion ... + p.productIterator.zipWithIndex.foreach { case (v, i) => + builder.append(i, v.asInstanceOf[Object], ois(i)) + } + } + + statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) + Iterator(builder.build()) + }.persist() + + // Put the table in the metastore. Let's use a fake DML statement. + HiveUtils.createTable(tableName, fields, manifests) + + // Force evaluate to put the data in memory. + SharkEnv.memoryMetadataManager.put(tableName, rdd) + rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + + // Gather the partition statistics. + SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) + } +} + + +object RDDTable { + + private type M[T] = ClassManifest[T] + private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + + def apply[T1: M, T2: M](rdd: RDD[(T1, T2)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2])) + } + + def apply[T1: M, T2: M, T3: M](rdd: RDD[(T1, T2, T3)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3])) + } + + def apply[T1: M, T2: M, T3: M, T4: M](rdd: RDD[(T1, T2, T3, T4)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M](rdd: RDD[(T1, T2, T3, T4, T5)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](rdd: RDD[(T1, T2, T3, T4, T5, T6)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + } +} diff --git a/src/main/scala/shark/memstore2/TablePartitionBuilder.scala b/src/main/scala/shark/memstore2/TablePartitionBuilder.scala index cdd2843d..8614c070 100644 --- a/src/main/scala/shark/memstore2/TablePartitionBuilder.scala +++ b/src/main/scala/shark/memstore2/TablePartitionBuilder.scala @@ -18,10 +18,10 @@ package shark.memstore2 import java.io.{DataInput, DataOutput} -import java.util.{List => JList} + +import scala.collection.JavaConversions._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.StructField import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.io.Writable @@ -33,19 +33,22 @@ import shark.memstore2.column.ColumnBuilder * partition of data into columnar format and to generate a TablePartition. */ class TablePartitionBuilder( - oi: StructObjectInspector, + ois: Seq[ObjectInspector], initialColumnSize: Int, - shouldCompress: Boolean = true) + shouldCompress: Boolean) extends Writable { - var numRows: Long = 0 - val fields: JList[_ <: StructField] = oi.getAllStructFieldRefs + def this(oi: StructObjectInspector, initialColumnSize: Int, shouldCompress: Boolean = true) = { + this(oi.getAllStructFieldRefs.map(_.getFieldObjectInspector), initialColumnSize, shouldCompress) + } + + private var numRows: Long = 0 - val columnBuilders = Array.tabulate[ColumnBuilder[_]](fields.size) { i => - val columnBuilder = ColumnBuilder.create(fields.get(i).getFieldObjectInspector, shouldCompress) + private val columnBuilders: Array[ColumnBuilder[_]] = ois.map { oi => + val columnBuilder = ColumnBuilder.create(oi, shouldCompress) columnBuilder.initialize(initialColumnSize) columnBuilder - } + }.toArray def incrementRowCount() { numRows += 1 @@ -57,7 +60,7 @@ class TablePartitionBuilder( def stats: TablePartitionStats = new TablePartitionStats(columnBuilders.map(_.stats), numRows) - def build: TablePartition = new TablePartition(numRows, columnBuilders.map(_.build)) + def build(): TablePartition = new TablePartition(numRows, columnBuilders.map(_.build())) // We don't use these, but want to maintain Writable interface for SerDe override def write(out: DataOutput) {} diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala new file mode 100644 index 00000000..fc929a31 --- /dev/null +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.util + +import java.util.{ArrayList => JavaArrayList, HashSet => JavaHashSet} +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} +import org.apache.hadoop.hive.ql.plan.{DDLWork, CreateTableDesc} +import org.apache.hadoop.hive.metastore.api.FieldSchema + +import shark.api.DataTypes +import org.apache.hadoop.hive.ql.exec.DDLTask +import org.apache.hadoop.hive.conf.HiveConf + + +private[shark] object HiveUtils { + + private val timestampManfiest = classManifest[java.sql.Timestamp] + private val stringManifest = classManifest[String] + + def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = m match { + case Manifest.Boolean => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case Manifest.Byte => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case Manifest.Short => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case Manifest.Int => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case Manifest.Long => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case Manifest.Float => PrimitiveObjectInspectorFactory.javaFloatObjectInspector + case Manifest.Double => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case Manifest.Unit => PrimitiveObjectInspectorFactory.javaVoidObjectInspector + case `timestampManfiest` => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case `stringManifest` => PrimitiveObjectInspectorFactory.javaStringObjectInspector + } + + /** + * Execute the create table DDL operation against Hive's metastore. + */ + def createTable(tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]]) { + val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => + new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") + } + + // Setup the create table descriptor with necessary information. + val desc = new CreateTableDesc() + desc.setTableName(tableName) + desc.setCols(new JavaArrayList[FieldSchema](schema)) + desc.setTblProps(Map("shark.cache" -> "heap")) + desc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") + desc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") + desc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) + desc.setNumBuckets(-1) + + // Execute the create table against the metastore. + val work = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], desc) + val task = new DDLTask + task.initialize(new HiveConf, null, null) + task.setWork(work) + task.execute(null) + } +} \ No newline at end of file From 147449f0cf9fbc44358341ff1a67e95f61872208 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 25 Aug 2013 22:41:04 -0700 Subject: [PATCH 162/331] Error handling for RDDTable. --- src/main/scala/shark/api/RDDTable.scala | 23 +++++++++++++++-------- src/main/scala/shark/util/HiveUtils.scala | 10 ++++++++-- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 3cb3e988..a03622c2 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -27,7 +27,10 @@ import spark.RDD class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { - def saveAsTable(tableName: String, fields: Seq[String]) { + def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { + require(fields.size == this.manifests.size, + "Number of column names != number of fields in the RDD.") + // Get a local copy of the manifests so we don't need to serialize this object. val manifests = this.manifests @@ -50,15 +53,19 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() - // Put the table in the metastore. Let's use a fake DML statement. - HiveUtils.createTable(tableName, fields, manifests) + // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. + if (HiveUtils.createTable(tableName, fields, manifests)) { + // Force evaluate to put the data in memory. + SharkEnv.memoryMetadataManager.put(tableName, rdd) + rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) - // Force evaluate to put the data in memory. - SharkEnv.memoryMetadataManager.put(tableName, rdd) - rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + // Gather the partition statistics. + SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) - // Gather the partition statistics. - SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) + true + } else { + false + } } } diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index fc929a31..d7542f90 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -52,7 +52,11 @@ private[shark] object HiveUtils { /** * Execute the create table DDL operation against Hive's metastore. */ - def createTable(tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]]) { + def createTable( + tableName: String, + columnNames: Seq[String], + columnTypes: Seq[ClassManifest[_]]): Boolean = + { val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") } @@ -72,6 +76,8 @@ private[shark] object HiveUtils { val task = new DDLTask task.initialize(new HiveConf, null, null) task.setWork(work) - task.execute(null) + + // Hive returns 0 if the create table command is executed successfully. + task.execute(null) == 0 } } \ No newline at end of file From fd1d56e25d0ecacb9c83a6ad5646afd0041b4f62 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 19 Sep 2013 14:45:36 -0700 Subject: [PATCH 163/331] New spark naming --- src/main/scala/shark/api/RDDTable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index a03622c2..8a3ce1bc 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import shark.SharkEnv import shark.memstore2.{TablePartitionStats, TablePartition, TablePartitionBuilder} import shark.util.HiveUtils -import spark.RDD +import org.apache.spark.rdd.RDD class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { From 05733ea5be5304fb748ebef88a4dd0651f1ccf52 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 18:46:53 -0700 Subject: [PATCH 164/331] Minor syntax changes to RDDTable. --- src/main/scala/shark/api/RDDTable.scala | 11 +++++------ .../scala/shark/execution/TableScanOperator.scala | 4 ++-- src/main/scala/shark/util/HiveUtils.scala | 9 +++++---- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 8a3ce1bc..a23e1632 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -53,24 +53,23 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() + val isDDLStatementSuccessful = HiveUtils.createTable(tableName, fields, manifests) + // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. - if (HiveUtils.createTable(tableName, fields, manifests)) { + if (isDDLStatementSuccessful) { // Force evaluate to put the data in memory. SharkEnv.memoryMetadataManager.put(tableName, rdd) rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) // Gather the partition statistics. SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) - - true - } else { - false } + return isDDLStatementSuccessful } } -object RDDTable { +object RDDToTable { private type M[T] = ClassManifest[T] private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index b822c64b..fc6ebd61 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -139,7 +139,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO throw(new QueryExecutionException("Cached table not found")) } logInfo("Loading table " + tableKey + " from Spark block manager") - createPrunedRdd(tableKey, rdd) + return createPrunedRdd(tableKey, rdd) } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { @@ -211,7 +211,7 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO rdd } - prunedRdd.mapPartitions { iter => + return prunedRdd.mapPartitions { iter => if (iter.hasNext) { val tablePartition = iter.next.asInstanceOf[TablePartition] tablePartition.prunedIterator(columnsUsed) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index d7542f90..6b6254c6 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -55,8 +55,7 @@ private[shark] object HiveUtils { def createTable( tableName: String, columnNames: Seq[String], - columnTypes: Seq[ClassManifest[_]]): Boolean = - { + columnTypes: Seq[ClassManifest[_]]): Boolean = { val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") } @@ -78,6 +77,8 @@ private[shark] object HiveUtils { task.setWork(work) // Hive returns 0 if the create table command is executed successfully. - task.execute(null) == 0 + val taskExecutionStatus: Int = task.execute(null) + + return taskExecutionStatus } -} \ No newline at end of file +} From 0f4f7e06d9bc3f42e540461f18b16ca405f68123 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 Sep 2013 20:28:09 -0700 Subject: [PATCH 165/331] Recognize Java primitive types in RDDTable. --- src/main/scala/shark/util/HiveUtils.scala | 28 +++++++++++------------ 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 6b6254c6..10197066 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -26,27 +26,27 @@ import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} import org.apache.hadoop.hive.ql.plan.{DDLWork, CreateTableDesc} import org.apache.hadoop.hive.metastore.api.FieldSchema -import shark.api.DataTypes +import shark.api.{DataType, DataTypes} import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.conf.HiveConf private[shark] object HiveUtils { - private val timestampManfiest = classManifest[java.sql.Timestamp] - private val stringManifest = classManifest[String] + def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = { + getJavaPrimitiveObjectInspector(DataTypes.fromManifest(m)) + } - def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = m match { - case Manifest.Boolean => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector - case Manifest.Byte => PrimitiveObjectInspectorFactory.javaByteObjectInspector - case Manifest.Short => PrimitiveObjectInspectorFactory.javaShortObjectInspector - case Manifest.Int => PrimitiveObjectInspectorFactory.javaIntObjectInspector - case Manifest.Long => PrimitiveObjectInspectorFactory.javaLongObjectInspector - case Manifest.Float => PrimitiveObjectInspectorFactory.javaFloatObjectInspector - case Manifest.Double => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector - case Manifest.Unit => PrimitiveObjectInspectorFactory.javaVoidObjectInspector - case `timestampManfiest` => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector - case `stringManifest` => PrimitiveObjectInspectorFactory.javaStringObjectInspector + def getJavaPrimitiveObjectInspector(t: DataType): PrimitiveObjectInspector = t match { + case DataTypes.BOOLEAN => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case DataTypes.TINYINT => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case DataTypes.SMALLINT => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case DataTypes.INT => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case DataTypes.BIGINT => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case DataTypes.FLOAT => PrimitiveObjectInspectorFactory.javaFloatObjectInspector + case DataTypes.DOUBLE => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case DataTypes.TIMESTAMP => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DataTypes.STRING => PrimitiveObjectInspectorFactory.javaStringObjectInspector } /** From 3840ec59115c1d353167e277e4a528be6d2071cc Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 20:49:19 -0700 Subject: [PATCH 166/331] Remove table from Shark and Hive metadata if the Spark job for RDD => Shark table RDD transformation fails. --- src/main/scala/shark/api/RDDTable.scala | 23 +++++++--- src/main/scala/shark/util/HiveUtils.scala | 54 ++++++++++++++++------- 2 files changed, 55 insertions(+), 22 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index a23e1632..0db90722 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -51,25 +51,36 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) Iterator(builder.build()) - }.persist() + }.persist(self.getStorageLevel) - val isDDLStatementSuccessful = HiveUtils.createTable(tableName, fields, manifests) + var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. - if (isDDLStatementSuccessful) { + if (isSucessfulCreateTable) { // Force evaluate to put the data in memory. SharkEnv.memoryMetadataManager.put(tableName, rdd) - rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + try { + rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + } catch { + case _ => { + // Intercept the exception thrown by SparkContext#runJob() and return silently. The + // exception message should be printed to the console by DDLTask#execute(). + HiveUtils.dropTableInHive(tableName) + // Drop the table entry from MemoryMetadataManager. + SharkEnv.unpersist(tableName) + isSucessfulCreateTable = false + } + } // Gather the partition statistics. SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) } - return isDDLStatementSuccessful + return isSucessfulCreateTable } } -object RDDToTable { +object RDDTable { private type M[T] = ClassManifest[T] private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 10197066..6b15f75b 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} -import org.apache.hadoop.hive.ql.plan.{DDLWork, CreateTableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLDesc, DDLWork, DropTableDesc} import org.apache.hadoop.hive.metastore.api.FieldSchema import shark.api.{DataType, DataTypes} @@ -52,7 +52,7 @@ private[shark] object HiveUtils { /** * Execute the create table DDL operation against Hive's metastore. */ - def createTable( + def createTableInHive( tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]]): Boolean = { @@ -61,24 +61,46 @@ private[shark] object HiveUtils { } // Setup the create table descriptor with necessary information. - val desc = new CreateTableDesc() - desc.setTableName(tableName) - desc.setCols(new JavaArrayList[FieldSchema](schema)) - desc.setTblProps(Map("shark.cache" -> "heap")) - desc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") - desc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") - desc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) - desc.setNumBuckets(-1) + val createTbleDesc = new CreateTableDesc() + createTbleDesc.setTableName(tableName) + createTbleDesc.setCols(new JavaArrayList[FieldSchema](schema)) + createTbleDesc.setTblProps(Map("shark.cache" -> "heap")) + createTbleDesc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") + createTbleDesc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") + createTbleDesc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) + createTbleDesc.setNumBuckets(-1) // Execute the create table against the metastore. - val work = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], desc) - val task = new DDLTask + val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], + new JavaHashSet[WriteEntity], + createTbleDesc) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + return (taskExecutionStatus == 0) + } + + def dropTableInHive( + tableName: String): Boolean = { + // Setup the drop table descriptor with necessary information. + val dropTblDesc = new DropTableDesc( + tableName, + false /* expectView. Should probably be named "isView".*/, + false /* ifExists */, + false /* stringPartitionColumns */) + + // Execute the drop table against the metastore. + val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], + new JavaHashSet[WriteEntity], + dropTblDesc) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + return (taskExecutionStatus == 0) + } + + def executeDDLTaskDirectly(ddlWork: DDLWork): Int = { + val task = new DDLTask() task.initialize(new HiveConf, null, null) - task.setWork(work) + task.setWork(ddlWork) // Hive returns 0 if the create table command is executed successfully. - val taskExecutionStatus: Int = task.execute(null) - - return taskExecutionStatus + return task.execute(null) } } From 9fcfe1b1ddf2ad8e8a551f73216843158c1077da Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 20:53:09 -0700 Subject: [PATCH 167/331] Merge remote-tracking branch 'ampshark/rddtable' into rddtable --- src/main/scala/shark/api/RDDTable.scala | 4 ++-- src/main/scala/shark/util/HiveUtils.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 0db90722..820ea65d 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -63,8 +63,8 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } catch { case _ => { - // Intercept the exception thrown by SparkContext#runJob() and return silently. The - // exception message should be printed to the console by DDLTask#execute(). + // Intercept the exception thrown by SparkContext#runJob() and handle it silently. The + // exception message should already be printed to the console by DDLTask#execute(). HiveUtils.dropTableInHive(tableName) // Drop the table entry from MemoryMetadataManager. SharkEnv.unpersist(tableName) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 6b15f75b..8f2f1473 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -83,7 +83,7 @@ private[shark] object HiveUtils { // Setup the drop table descriptor with necessary information. val dropTblDesc = new DropTableDesc( tableName, - false /* expectView. Should probably be named "isView".*/, + false /* expectView */, false /* ifExists */, false /* stringPartitionColumns */) From 98ae5d4676dfd04e444cd5d7f3d620699f2b20a6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 19 Sep 2013 21:07:58 -0700 Subject: [PATCH 168/331] Don't persist new Shark RDD with same storage level as parent. --- src/main/scala/shark/api/RDDTable.scala | 2 +- src/main/scala/shark/util/HiveUtils.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 820ea65d..31070873 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -51,7 +51,7 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) Iterator(builder.build()) - }.persist(self.getStorageLevel) + }.persist() var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 8f2f1473..5a9e686e 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} -import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLDesc, DDLWork, DropTableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} import org.apache.hadoop.hive.metastore.api.FieldSchema import shark.api.{DataType, DataTypes} From 54e6f05d5dde6a9e9975e526a5c09caf50e323a0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 25 Oct 2013 18:18:55 -0700 Subject: [PATCH 169/331] Increase # of types in RDDTable#apply() to 22, the limit for Scala's tuple length. --- src/main/scala/shark/api/RDDTable.scala | 165 +++++++++++------- .../scala/shark/api/RDDTableFunctions.scala | 81 +++++++++ 2 files changed, 185 insertions(+), 61 deletions(-) create mode 100644 src/main/scala/shark/api/RDDTableFunctions.scala diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 31070873..44978085 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -17,69 +17,9 @@ package shark.api -import scala.collection.mutable.ArrayBuffer - -import shark.SharkEnv -import shark.memstore2.{TablePartitionStats, TablePartition, TablePartitionBuilder} -import shark.util.HiveUtils import org.apache.spark.rdd.RDD -class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { - - def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { - require(fields.size == this.manifests.size, - "Number of column names != number of fields in the RDD.") - - // Get a local copy of the manifests so we don't need to serialize this object. - val manifests = this.manifests - - val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) - - // Create the RDD object. - val rdd = self.mapPartitionsWithIndex { case(partitionIndex, iter) => - val ois = manifests.map(HiveUtils.getJavaPrimitiveObjectInspector) - val builder = new TablePartitionBuilder(ois, 1000000, shouldCompress = false) - - for (p <- iter) { - builder.incrementRowCount() - // TODO: this is not the most efficient code to do the insertion ... - p.productIterator.zipWithIndex.foreach { case (v, i) => - builder.append(i, v.asInstanceOf[Object], ois(i)) - } - } - - statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) - Iterator(builder.build()) - }.persist() - - var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) - - // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. - if (isSucessfulCreateTable) { - // Force evaluate to put the data in memory. - SharkEnv.memoryMetadataManager.put(tableName, rdd) - try { - rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) - } catch { - case _ => { - // Intercept the exception thrown by SparkContext#runJob() and handle it silently. The - // exception message should already be printed to the console by DDLTask#execute(). - HiveUtils.dropTableInHive(tableName) - // Drop the table entry from MemoryMetadataManager. - SharkEnv.unpersist(tableName) - isSucessfulCreateTable = false - } - } - - // Gather the partition statistics. - SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) - } - return isSucessfulCreateTable - } -} - - object RDDTable { private type M[T] = ClassManifest[T] @@ -129,4 +69,107 @@ object RDDTable { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) } -} + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) + } + + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T:12: M, T:13]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16])) + } + + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17])) + } + + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, + T18)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + } + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) + } + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) + } + } + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M]( + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19 + T20, T21, T22)]) = { + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) + } + } \ No newline at end of file diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala new file mode 100644 index 00000000..2ae0da3d --- /dev/null +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -0,0 +1,81 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +import scala.collection.mutable.ArrayBuffer + +import shark.SharkEnv +import shark.memstore2.{TablePartitionStats, TablePartition, TablePartitionBuilder} +import shark.util.HiveUtils + +import org.apache.spark.rdd.RDD + + +class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { + + def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { + require(fields.size == this.manifests.size, + "Number of column names != number of fields in the RDD.") + + // Get a local copy of the manifests so we don't need to serialize this object. + val manifests = this.manifests + + val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) + + // Create the RDD object. + val rdd = self.mapPartitionsWithIndex { case(partitionIndex, iter) => + val ois = manifests.map(HiveUtils.getJavaPrimitiveObjectInspector) + val builder = new TablePartitionBuilder(ois, 1000000, shouldCompress = false) + + for (p <- iter) { + builder.incrementRowCount() + // TODO: this is not the most efficient code to do the insertion ... + p.productIterator.zipWithIndex.foreach { case (v, i) => + builder.append(i, v.asInstanceOf[Object], ois(i)) + } + } + + statsAcc += Tuple2(partitionIndex, builder.asInstanceOf[TablePartitionBuilder].stats) + Iterator(builder.build()) + }.persist() + + var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) + + // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. + if (isSucessfulCreateTable) { + // Force evaluate to put the data in memory. + SharkEnv.memoryMetadataManager.put(tableName, rdd) + try { + rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + } catch { + case _ => { + // Intercept the exception thrown by SparkContext#runJob() and handle it silently. The + // exception message should already be printed to the console by DDLTask#execute(). + HiveUtils.dropTableInHive(tableName) + // Drop the table entry from MemoryMetadataManager. + SharkEnv.unpersist(tableName) + isSucessfulCreateTable = false + } + } + + // Gather the partition statistics. + SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) + } + return isSucessfulCreateTable + } +} From e57e14bb32415842d6c74bce19ac867aef5761a0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 25 Oct 2013 18:19:15 -0700 Subject: [PATCH 170/331] Bump up version to Spark 0.9. --- project/SharkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 088e8fc0..53d85c93 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -26,9 +26,9 @@ import scala.util.Properties.{ envOrNone => env } object SharkBuild extends Build { // Shark version - val SHARK_VERSION = "0.8.0-SNAPSHOT" + val SHARK_VERSION = "0.9.0-SNAPSHOT" - val SPARK_VERSION = "0.8.0-SNAPSHOT" + val SPARK_VERSION = "0.9.0-incubating-SNAPSHOT" val SCALA_VERSION = "2.9.3" From 3c85f9a5296441db3ccf43882d9c8cb18f14783f Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 25 Oct 2013 18:32:18 -0700 Subject: [PATCH 171/331] Fix compilation bug for RDDTable. --- src/main/scala/shark/api/RDDTable.scala | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 44978085..cfa40f2e 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -84,7 +84,7 @@ object RDDTable { def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T:12: M, T:13]( + T13: M]( rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], @@ -135,13 +135,12 @@ object RDDTable { } def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M]( + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M]( rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) - } + new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) } def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, @@ -151,7 +150,6 @@ object RDDTable { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) - } } def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, @@ -161,15 +159,14 @@ object RDDTable { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) - } } def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19 - T20, T21, T22)]) = { + rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22)]) = { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) - } - } \ No newline at end of file + } +} From ed7ff318419814daee53980e3d748a6f914223f4 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 25 Oct 2013 19:47:16 -0700 Subject: [PATCH 172/331] Tiny indentation fix to RDDTable.scala --- src/main/scala/shark/api/RDDTable.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index cfa40f2e..595ea31f 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -84,16 +84,14 @@ object RDDTable { def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { + T13: M](rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13])) } def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { + T13: M, T14: M](rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14])) @@ -115,7 +113,6 @@ object RDDTable { m[T13], m[T14], m[T15], m[T16])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M]( rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]) = { @@ -124,7 +121,6 @@ object RDDTable { m[T13], m[T14], m[T15], m[T16], m[T17])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M]( rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, From f4ef44168ca9a8ae330e2dcdb20efe18695ac0f7 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 26 Oct 2013 17:34:24 -0700 Subject: [PATCH 173/331] Fix errors from the rddtable merge. --- src/main/scala/shark/api/RDDTableFunctions.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 2ae0da3d..837bad1c 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -20,7 +20,7 @@ package shark.api import scala.collection.mutable.ArrayBuffer import shark.SharkEnv -import shark.memstore2.{TablePartitionStats, TablePartition, TablePartitionBuilder} +import shark.memstore2.{CacheType, TablePartitionStats, TablePartition, TablePartitionBuilder} import shark.util.HiveUtils import org.apache.spark.rdd.RDD @@ -58,9 +58,12 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. if (isSucessfulCreateTable) { - // Force evaluate to put the data in memory. - SharkEnv.memoryMetadataManager.put(tableName, rdd) + // Create an entry in the MemoryMetadataManager. + val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( + tableName, CacheType.HEAP, rdd.getStorageLevel) + newTable.tableRDD = rdd try { + // Force evaluate to put the data in memory. rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } catch { case _ => { @@ -68,7 +71,7 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { // exception message should already be printed to the console by DDLTask#execute(). HiveUtils.dropTableInHive(tableName) // Drop the table entry from MemoryMetadataManager. - SharkEnv.unpersist(tableName) + SharkEnv.memoryMetadataManager.removeTable(tableName) isSucessfulCreateTable = false } } From 797f2be907227a9f06e78d353d52500d991d5aba Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 18 Sep 2013 11:14:44 +0800 Subject: [PATCH 174/331] Remove hive operators Conflicts: src/main/scala/shark/execution/FileSinkOperator.scala src/main/scala/shark/execution/TableScanOperator.scala src/main/scala/shark/parse/SharkSemanticAnalyzer.scala --- .../shark/execution/ExplainTaskHelper.java | 8 +- .../execution/ReduceSinkOperatorHelper.java | 31 ---- .../shark/execution/CommonJoinOperator.scala | 31 +++- .../shark/execution/ExtractOperator.scala | 21 ++- .../shark/execution/FilterOperator.scala | 6 +- .../shark/execution/ForwardOperator.scala | 3 +- .../GroupByPostShuffleOperator.scala | 27 ++-- .../execution/GroupByPreShuffleOperator.scala | 56 +++++-- .../shark/execution/HiveTopOperator.scala | 101 ------------ .../scala/shark/execution/JoinOperator.scala | 10 +- src/main/scala/shark/execution/JoinUtil.scala | 19 ++- .../LateralViewForwardOperator.scala | 3 +- .../execution/LateralViewJoinOperator.scala | 34 +++- .../scala/shark/execution/LimitOperator.scala | 9 +- .../shark/execution/MapJoinOperator.scala | 31 +++- src/main/scala/shark/execution/Operator.scala | 151 ++++++++++++++++-- .../shark/execution/OperatorFactory.scala | 87 +++++----- .../shark/execution/ReduceSinkOperator.scala | 55 +++---- .../shark/execution/ReduceSinkTableDesc.scala | 33 ++++ .../shark/execution/ScriptOperator.scala | 34 ++-- .../shark/execution/SelectOperator.scala | 25 ++- .../scala/shark/execution/SparkTask.scala | 25 --- .../shark/execution/TableScanOperator.scala | 70 ++++---- .../shark/execution/TerminalOperator.scala | 11 +- .../scala/shark/execution/UDTFOperator.scala | 13 +- .../scala/shark/execution/UnionOperator.scala | 27 ++-- src/main/scala/shark/execution/package.scala | 4 +- .../OperatorSerializationWrapper.scala | 6 +- .../shark/parse/SharkSemanticAnalyzer.scala | 20 +-- 29 files changed, 541 insertions(+), 410 deletions(-) delete mode 100755 src/main/java/shark/execution/ReduceSinkOperatorHelper.java delete mode 100755 src/main/scala/shark/execution/HiveTopOperator.scala create mode 100644 src/main/scala/shark/execution/ReduceSinkTableDesc.scala diff --git a/src/main/java/shark/execution/ExplainTaskHelper.java b/src/main/java/shark/execution/ExplainTaskHelper.java index b85bce50..933ebb62 100755 --- a/src/main/java/shark/execution/ExplainTaskHelper.java +++ b/src/main/java/shark/execution/ExplainTaskHelper.java @@ -58,11 +58,11 @@ public static void outputPlan(Serializable work, PrintStream out, boolean extend // conf and then // the children if (work instanceof shark.execution.Operator) { - shark.execution.Operator> operator = - (shark.execution.Operator>) work; + shark.execution.Operator operator = + (shark.execution.Operator) work; out.println(indentString(indent) + "**" + operator.getClass().getName()); - if (operator.hiveOp().getConf() != null) { - outputPlan(operator.hiveOp().getConf(), out, extended, indent); + if (operator.desc() != null) { + outputPlan(operator.desc(), out, extended, indent); } if (operator.parentOperators() != null) { for (shark.execution.Operator op : operator.parentOperatorsAsJavaList()) { diff --git a/src/main/java/shark/execution/ReduceSinkOperatorHelper.java b/src/main/java/shark/execution/ReduceSinkOperatorHelper.java deleted file mode 100755 index e7f915fd..00000000 --- a/src/main/java/shark/execution/ReduceSinkOperatorHelper.java +++ /dev/null @@ -1,31 +0,0 @@ -package shark.execution; - -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.ql.metadata.HiveException; - -import java.util.List; - - -/** - * This class is used because we cannot call protected static methods from - * Scala. - */ -@SuppressWarnings("serial") -public class ReduceSinkOperatorHelper extends ReduceSinkOperator { - - public static StructObjectInspector initEvaluatorsAndReturnStruct( - ExprNodeEvaluator[] evals, List> distinctColIndices, - List outputColNames, int length, ObjectInspector rowInspector) - throws HiveException { - - return ReduceSinkOperator.initEvaluatorsAndReturnStruct( - evals, - distinctColIndices, - outputColNames, - length, - rowInspector); - } -} diff --git a/src/main/scala/shark/execution/CommonJoinOperator.scala b/src/main/scala/shark/execution/CommonJoinOperator.scala index a081ade5..5e7ce572 100755 --- a/src/main/scala/shark/execution/CommonJoinOperator.scala +++ b/src/main/scala/shark/execution/CommonJoinOperator.scala @@ -17,7 +17,7 @@ package shark.execution -import java.util.{HashMap => JavaHashMap, List => JavaList} +import java.util.{HashMap => JavaHashMap, List => JavaList, ArrayList =>JavaArrayList} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ @@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.{JoinUtil => HiveJoinUtil} import org.apache.hadoop.hive.ql.plan.{ExprNodeDesc, JoinCondDesc, JoinDesc, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, PrimitiveObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.SparkContext.rddToPairRDDFunctions @@ -37,10 +38,9 @@ import org.apache.spark.SparkContext.rddToPairRDDFunctions import shark.SharkConfVars -abstract class CommonJoinOperator[JOINDESCTYPE <: JoinDesc, T <: HiveCommonJoinOperator[JOINDESCTYPE]] - extends NaryOperator[T] { +abstract class CommonJoinOperator[T <: JoinDesc] extends NaryOperator[T] { - @BeanProperty var conf: JOINDESCTYPE = _ + @BeanProperty var conf: T = _ // Order in which the results should be output. @BeanProperty var order: Array[java.lang.Byte] = _ // condn determines join property (left, right, outer joins). @@ -62,8 +62,11 @@ abstract class CommonJoinOperator[JOINDESCTYPE <: JoinDesc, T <: HiveCommonJoinO @transient var noOuterJoin: Boolean = _ override def initializeOnMaster() { - conf = hiveOp.getConf() - + super.initializeOnMaster() + conf = desc + // TODO currently remove the join filter + conf.getFilters().clear() + order = conf.getTagOrder() joinConditions = conf.getConds() numTables = parentOperators.size @@ -91,6 +94,19 @@ abstract class CommonJoinOperator[JOINDESCTYPE <: JoinDesc, T <: HiveCommonJoinO joinValuesStandardObjectInspectors = HiveJoinUtil.getStandardObjectInspectors( joinValuesObjectInspectors, CommonJoinOperator.NOTSKIPBIGTABLE) } + + // copied from the org.apache.hadoop.hive.ql.exec.CommonJoinOperator + override def outputObjectInspector() = { + var structFieldObjectInspectors = new JavaArrayList[ObjectInspector]() + for (alias <- order) { + var oiList = joinValuesStandardObjectInspectors.get(alias) + structFieldObjectInspectors.addAll(oiList) + } + + ObjectInspectorFactory.getStandardStructObjectInspector( + conf.getOutputColumnNames(), + structFieldObjectInspectors) + } } @@ -202,6 +218,9 @@ object CommonJoinOperator { */ def isFiltered(row: Any, filters: JavaList[ExprNodeEvaluator], ois: JavaList[ObjectInspector]) : Boolean = { + // if no filter, then will not be filtered + if(filters == null || ois == null) return false + var ret: java.lang.Boolean = false var j = 0 while (j < filters.size) { diff --git a/src/main/scala/shark/execution/ExtractOperator.scala b/src/main/scala/shark/execution/ExtractOperator.scala index 1f64a3e4..6f7c3467 100755 --- a/src/main/scala/shark/execution/ExtractOperator.scala +++ b/src/main/scala/shark/execution/ExtractOperator.scala @@ -18,18 +18,19 @@ package shark.execution import scala.reflect.BeanProperty - import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} import org.apache.hadoop.hive.ql.exec.{ExtractOperator => HiveExtractOperator} import org.apache.hadoop.hive.ql.plan.{ExtractDesc, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.io.BytesWritable - import org.apache.spark.rdd.RDD +import shark.SharkConfVars +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector -class ExtractOperator extends UnaryOperator[HiveExtractOperator] with HiveTopOperator { +class ExtractOperator extends UnaryOperator[ExtractDesc] + with ReduceSinkTableDesc { @BeanProperty var conf: ExtractDesc = _ @BeanProperty var valueTableDesc: TableDesc = _ @@ -39,18 +40,28 @@ class ExtractOperator extends UnaryOperator[HiveExtractOperator] with HiveTopOpe @transient var valueDeser: Deserializer = _ override def initializeOnMaster() { - conf = hiveOp.getConf() + super.initializeOnMaster() + + conf = desc localHconf = super.hconf - valueTableDesc = keyValueTableDescs.values.head._2 + valueTableDesc = keyValueDescs().head._2._2 } override def initializeOnSlave() { + super.initializeOnSlave() + eval = ExprNodeEvaluatorFactory.get(conf.getCol) eval.initialize(objectInspector) valueDeser = valueTableDesc.getDeserializerClass().newInstance() valueDeser.initialize(localHconf, valueTableDesc.getProperties()) } + override def outputObjectInspector() = { + var soi = objectInspectors(0).asInstanceOf[StructObjectInspector] + // take the value part + soi.getAllStructFieldRefs().get(1).getFieldObjectInspector() + } + override def preprocessRdd(rdd: RDD[_]): RDD[_] = { // TODO: hasOrder and limit should really be made by optimizer. val hasOrder = parentOperator match { diff --git a/src/main/scala/shark/execution/FilterOperator.scala b/src/main/scala/shark/execution/FilterOperator.scala index 2548bb59..643c062f 100755 --- a/src/main/scala/shark/execution/FilterOperator.scala +++ b/src/main/scala/shark/execution/FilterOperator.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.plan.FilterDesc import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector -class FilterOperator extends UnaryOperator[HiveFilterOperator] { +class FilterOperator extends UnaryOperator[FilterDesc] { @transient var conditionEvaluator: ExprNodeEvaluator = _ @transient var conditionInspector: PrimitiveObjectInspector = _ @@ -35,7 +35,9 @@ class FilterOperator extends UnaryOperator[HiveFilterOperator] { @BeanProperty var conf: FilterDesc = _ override def initializeOnMaster() { - conf = hiveOp.getConf() + super.initializeOnMaster() + + conf = desc } override def initializeOnSlave() { diff --git a/src/main/scala/shark/execution/ForwardOperator.scala b/src/main/scala/shark/execution/ForwardOperator.scala index a4b488ae..e41d8f61 100755 --- a/src/main/scala/shark/execution/ForwardOperator.scala +++ b/src/main/scala/shark/execution/ForwardOperator.scala @@ -20,9 +20,10 @@ package shark.execution import org.apache.hadoop.hive.ql.exec.{ForwardOperator => HiveForwardOperator} import org.apache.spark.rdd.RDD +import org.apache.hadoop.hive.ql.plan.ForwardDesc -class ForwardOperator extends UnaryOperator[HiveForwardOperator] { +class ForwardOperator extends UnaryOperator[ForwardDesc] { override def execute(): RDD[_] = executeParents().head._2 diff --git a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala index c180fa34..94748161 100755 --- a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala @@ -19,16 +19,13 @@ package org.apache.hadoop.hive.ql.exec // Put this file in Hive's exec package to access package level visible fields and methods. import java.util.{ArrayList => JArrayList, HashMap => JHashMap, HashSet => JHashSet, Set => JSet} - import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.BeanProperty - import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.plan.{ExprNodeColumnDesc, TableDesc} import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorUtils, - StandardStructObjectInspector, StructObjectInspector, UnionObject} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorUtils,StandardStructObjectInspector, StructObjectInspector, UnionObject} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils @@ -44,7 +41,8 @@ import shark.execution.serialization.OperatorSerializationWrapper // The final phase of group by. // TODO(rxin): For multiple distinct aggregations, use sort-based shuffle. -class GroupByPostShuffleOperator extends GroupByPreShuffleOperator with HiveTopOperator { +class GroupByPostShuffleOperator extends GroupByPreShuffleOperator + with ReduceSinkTableDesc { @BeanProperty var keyTableDesc: TableDesc = _ @BeanProperty var valueTableDesc: TableDesc = _ @@ -64,16 +62,9 @@ class GroupByPostShuffleOperator extends GroupByPreShuffleOperator with HiveTopO @transient val distinctHashSets = new JHashMap[Int, JArrayList[JHashSet[KeyWrapper]]]() @transient var unionExprEvaluator: ExprNodeEvaluator = _ - override def initializeOnMaster() { - super.initializeOnMaster() - keyTableDesc = keyValueTableDescs.values.head._1 - valueTableDesc = keyValueTableDescs.values.head._2 - initializeOnSlave() - } + override def createLocals() { + super.createLocals() - override def initializeOnSlave() { - - super.initializeOnSlave() // Initialize unionExpr. KEY has union field as the last field if there are distinct aggrs. unionExprEvaluator = initializeUnionExprEvaluator(rowInspector) @@ -90,6 +81,14 @@ class GroupByPostShuffleOperator extends GroupByPreShuffleOperator with HiveTopO valueSer1 = valueTableDesc.getDeserializerClass.newInstance() valueSer1.initialize(null, valueTableDesc.getProperties()) } + + override def createRemotes() { + super.createRemotes() + + var kvd = keyValueDescs() + keyTableDesc = kvd.head._2._1 + valueTableDesc = kvd.head._2._2 + } private def initializeKeyWrapperFactories() { distinctKeyAggrs.keySet.iterator.foreach { unionId => diff --git a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala index 055f682a..c2275f59 100755 --- a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala @@ -21,10 +21,10 @@ package org.apache.hadoop.hive.ql.exec import java.util.{ArrayList => JArrayList, HashMap => JHashMap} import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer import scala.reflect.BeanProperty import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.exec.{GroupByOperator => HiveGroupByOperator} import org.apache.hadoop.hive.ql.plan.{AggregationDesc, ExprNodeDesc, GroupByDesc} import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer @@ -38,7 +38,7 @@ import shark.execution.UnaryOperator /** * The pre-shuffle group by operator responsible for map side aggregations. */ -class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { +class GroupByPreShuffleOperator extends UnaryOperator[GroupByDesc] { @BeanProperty var conf: GroupByDesc = _ @BeanProperty var minReductionHashAggr: Float = _ @@ -49,7 +49,7 @@ class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { // The aggregation functions. @transient var aggregationEvals: Array[GenericUDAFEvaluator] = _ - + @transient var aggregationObjectInspectors: Array[ObjectInspector] = _ // Key fields to be grouped. @transient var keyFields: Array[ExprNodeEvaluator] = _ // A struct object inspector composing of all the fields. @@ -60,20 +60,16 @@ class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { @transient var aggregationParameterStandardObjectInspectors: Array[Array[ObjectInspector]] = _ @transient var aggregationIsDistinct: Array[Boolean] = _ + @transient var currentKeyObjectInspectors: Array[ObjectInspector] = _ - override def initializeOnMaster() { - conf = hiveOp.getConf() - minReductionHashAggr = hconf.get(HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION.varname).toFloat - numRowsCompareHashAggr = hconf.get(HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL.varname).toInt - } - - override def initializeOnSlave() { + def createLocals() { aggregationEvals = conf.getAggregators.map(_.getGenericUDAFEvaluator).toArray aggregationIsDistinct = conf.getAggregators.map(_.getDistinct).toArray rowInspector = objectInspector.asInstanceOf[StructObjectInspector] keyFields = conf.getKeys().map(k => ExprNodeEvaluatorFactory.get(k)).toArray val keyObjectInspectors: Array[ObjectInspector] = keyFields.map(k => k.initialize(rowInspector)) - val currentKeyObjectInspectors = keyObjectInspectors.map { k => + currentKeyObjectInspectors = + keyObjectInspectors.map { k => ObjectInspectorUtils.getStandardObjectInspector(k, ObjectInspectorCopyOption.WRITABLE) } @@ -96,6 +92,12 @@ class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { aggregationParameterObjectInspectors(pair._2)) } + aggregationObjectInspectors = + Array.tabulate[ObjectInspector](aggregationEvals.length) { i=> + var mode = conf.getAggregators()(i).getMode() + aggregationEvals(i).init(mode, aggregationParameterObjectInspectors(i)) + } + val keyFieldNames = conf.getOutputColumnNames.slice(0, keyFields.length) val totalFields = keyFields.length + aggregationEvals.length val keyois = new JArrayList[ObjectInspector](totalFields) @@ -105,7 +107,39 @@ class GroupByPreShuffleOperator extends UnaryOperator[HiveGroupByOperator] { keyFactory = new KeyWrapperFactory(keyFields, keyObjectInspectors, currentKeyObjectInspectors) } + + def createRemotes() { + conf = desc + minReductionHashAggr = hconf.get(HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION.varname).toFloat + numRowsCompareHashAggr = hconf.get(HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL.varname).toInt + } + + override def initializeOnMaster() { + super.initializeOnMaster() + + createRemotes() + createLocals() + } + + override def initializeOnSlave() { + super.initializeOnSlave() + createLocals() + } + + // copied from the org.apache.hadoop.hive.ql.exec.GroupByOperator + override def outputObjectInspector() = { + var totalFields = keyFields.length + aggregationEvals.length + + var ois = new ArrayBuffer[ObjectInspector](totalFields) + ois.++=(currentKeyObjectInspectors) + ois.++=(aggregationObjectInspectors) + var fieldNames = conf.getOutputColumnNames() + + import scala.collection.JavaConversions._ + ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, ois.toList) + } + override def processPartition(split: Int, iter: Iterator[_]) = { logDebug("Running Pre-Shuffle Group-By") var numRowsInput = 0 diff --git a/src/main/scala/shark/execution/HiveTopOperator.scala b/src/main/scala/shark/execution/HiveTopOperator.scala deleted file mode 100755 index d9f611f4..00000000 --- a/src/main/scala/shark/execution/HiveTopOperator.scala +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.execution - -import org.apache.hadoop.hive.ql.metadata.HiveException -import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector - -import shark.LogHelper - - -/** - * Operators that are top operators in Hive stages. This includes TableScan and - * everything that can come after ReduceSink. Note that they might have multiple - * upstream operators (multiple parents). - */ -trait HiveTopOperator extends LogHelper { - self: Operator[_ <: HiveOperator] => - - /** - * Stores the input object inspectors. This is passed down by either the - * upstream operators (i.e. ReduceSink) or in the case of TableScan, passed - * by the init code in SparkTask. - */ - @transient - val inputObjectInspectors = new scala.collection.mutable.HashMap[Int, ObjectInspector] - - /** - * Stores the deser for operators downstream from ReduceSink. This is set by - * ReduceSink.initializeDownStreamHiveOperators(). - */ - @transient - val keyValueTableDescs = new scala.collection.mutable.HashMap[Int, (TableDesc, TableDesc)] - - /** - * Initialize the Hive operator when all input object inspectors are ready. - */ - def initializeHiveTopOperator() { - logDebug("Started executing " + self + " initializeHiveTopOperator()") - - // Call initializeDownStreamHiveOperators() of upstream operators that are - // ReduceSink so we can get the proper input object inspectors and serdes. - val reduceSinkParents = self.parentOperators.filter(_.isInstanceOf[ReduceSinkOperator]) - reduceSinkParents.foreach { parent => - parent.asInstanceOf[ReduceSinkOperator].initializeDownStreamHiveOperator() - logDebug("parent : " + parent) - } - - // Only do initialize if all our input inspectors are ready. We use > - // instead of == since TableScan doesn't have parents, but have an object - // inspector. If == is used, table scan is skipped. - assert(inputObjectInspectors.size >= reduceSinkParents.size, - println("# input object inspectors (%d) < # reduce sink parent operators (%d)".format( - inputObjectInspectors.size, reduceSinkParents.size))) - - val objectInspectorArray = { - // Special case for single object inspector (non join case) because the - // joinTag is -1. - if (inputObjectInspectors.size == 1) { - Array(inputObjectInspectors.values.head) - } else { - val arr = new Array[ObjectInspector](inputObjectInspectors.size) - inputObjectInspectors foreach { case (tag, inspector) => arr(tag) = inspector } - arr - } - } - - if (objectInspectorArray.size > 0) { - // Initialize the hive operators. This init propagates downstream. - logDebug("Executing " + self.hiveOp + ".initialize()") - self.hiveOp.initialize(hconf, objectInspectorArray) - } - - logDebug("Finished executing " + self + " initializeHiveTopOperator()") - } - - def setInputObjectInspector(tag: Int, objectInspector: ObjectInspector) { - inputObjectInspectors.put(tag, objectInspector) - } - - def setKeyValueTableDescs(tag: Int, descs: (TableDesc, TableDesc)) { - keyValueTableDescs.put(tag, descs) - } - -} - diff --git a/src/main/scala/shark/execution/JoinOperator.scala b/src/main/scala/shark/execution/JoinOperator.scala index a641a264..764eeaa5 100755 --- a/src/main/scala/shark/execution/JoinOperator.scala +++ b/src/main/scala/shark/execution/JoinOperator.scala @@ -36,8 +36,7 @@ import org.apache.spark.rdd.RDD import shark.execution.serialization.OperatorSerializationWrapper -class JoinOperator extends CommonJoinOperator[JoinDesc, HiveJoinOperator] - with HiveTopOperator { +class JoinOperator extends CommonJoinOperator[JoinDesc] with ReduceSinkTableDesc { @BeanProperty var valueTableDescMap: JHashMap[Int, TableDesc] = _ @BeanProperty var keyTableDesc: TableDesc = _ @@ -48,9 +47,10 @@ class JoinOperator extends CommonJoinOperator[JoinDesc, HiveJoinOperator] override def initializeOnMaster() { super.initializeOnMaster() + var descs = keyValueDescs() valueTableDescMap = new JHashMap[Int, TableDesc] - valueTableDescMap ++= keyValueTableDescs.map { case(tag, kvdescs) => (tag, kvdescs._2) } - keyTableDesc = keyValueTableDescs.head._2._1 + valueTableDescMap ++= descs.map { case(tag, kvdescs) => (tag, kvdescs._2) } + keyTableDesc = descs.head._2._1 // Call initializeOnSlave to initialize the join filters, etc. initializeOnSlave() @@ -113,7 +113,7 @@ class JoinOperator extends CommonJoinOperator[JoinDesc, HiveJoinOperator] op.initializeOnSlave() val writable = new BytesWritable - val nullSafes = op.conf.getNullSafes() + val nullSafes = conf.getNullSafes() val cp = new CartesianProduct[Any](op.numTables) diff --git a/src/main/scala/shark/execution/JoinUtil.scala b/src/main/scala/shark/execution/JoinUtil.scala index 2fb22731..3fdf1806 100644 --- a/src/main/scala/shark/execution/JoinUtil.scala +++ b/src/main/scala/shark/execution/JoinUtil.scala @@ -56,15 +56,18 @@ object JoinUtil { noOuterJoin: Boolean): Array[AnyRef] = { val isFiltered: Boolean = { - Range(0, filters.size()).exists { x => - val cond = filters.get(x).evaluate(row) - val result = Option[AnyRef]( - filtersOI.get(x).asInstanceOf[PrimitiveOI].getPrimitiveJavaObject(cond)) - result match { - case Some(u) => u.asInstanceOf[Boolean].unary_! - case None => true + if(filters == null) + false + else + Range(0, filters.size()).exists { x => + val cond = filters.get(x).evaluate(row) + val result = Option[AnyRef]( + filtersOI.get(x).asInstanceOf[PrimitiveOI].getPrimitiveJavaObject(cond)) + result match { + case Some(u) => u.asInstanceOf[Boolean].unary_! + case None => true + } } - } } val size = valueFields.size val a = new Array[AnyRef](size) diff --git a/src/main/scala/shark/execution/LateralViewForwardOperator.scala b/src/main/scala/shark/execution/LateralViewForwardOperator.scala index 65efd679..ec497d23 100755 --- a/src/main/scala/shark/execution/LateralViewForwardOperator.scala +++ b/src/main/scala/shark/execution/LateralViewForwardOperator.scala @@ -18,11 +18,12 @@ package shark.execution import org.apache.hadoop.hive.ql.exec.{LateralViewForwardOperator => HiveLateralViewForwardOperator} +import org.apache.hadoop.hive.ql.plan.LateralViewForwardDesc import org.apache.spark.rdd.RDD -class LateralViewForwardOperator extends UnaryOperator[HiveLateralViewForwardOperator] { +class LateralViewForwardOperator extends UnaryOperator[LateralViewForwardDesc] { override def execute(): RDD[_] = executeParents().head._2 diff --git a/src/main/scala/shark/execution/LateralViewJoinOperator.scala b/src/main/scala/shark/execution/LateralViewJoinOperator.scala index 55164348..cba93e43 100755 --- a/src/main/scala/shark/execution/LateralViewJoinOperator.scala +++ b/src/main/scala/shark/execution/LateralViewJoinOperator.scala @@ -19,18 +19,18 @@ package shark.execution import java.nio.ByteBuffer import java.util.ArrayList - import scala.collection.JavaConversions._ import scala.reflect.BeanProperty - import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} import org.apache.hadoop.hive.ql.exec.{LateralViewJoinOperator => HiveLateralViewJoinOperator} import org.apache.hadoop.hive.ql.plan.SelectDesc +import org.apache.hadoop.hive.ql.plan.LateralViewJoinDesc import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} - import org.apache.spark.rdd.RDD import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} +import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory /** @@ -39,7 +39,7 @@ import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} * Hive handles this by having two branches in its plan, then joining their output (see diagram in * LateralViewJoinOperator.java). We put all the explode logic here instead. */ -class LateralViewJoinOperator extends NaryOperator[HiveLateralViewJoinOperator] { +class LateralViewJoinOperator extends NaryOperator[LateralViewJoinDesc] { @BeanProperty var conf: SelectDesc = _ @BeanProperty var lvfOp: LateralViewForwardOperator = _ @@ -51,9 +51,10 @@ class LateralViewJoinOperator extends NaryOperator[HiveLateralViewJoinOperator] @transient var fieldOis: StructObjectInspector = _ override def initializeOnMaster() { + super.initializeOnMaster() // Get conf from Select operator beyond UDTF Op to get eval() conf = parentOperators.filter(_.isInstanceOf[UDTFOperator]).head - .parentOperators.head.asInstanceOf[SelectOperator].hiveOp.getConf() + .parentOperators.head.asInstanceOf[SelectOperator].desc udtfOp = parentOperators.filter(_.isInstanceOf[UDTFOperator]).head.asInstanceOf[UDTFOperator] udtfOIString = KryoSerializerToString.serialize(udtfOp.objectInspectors) @@ -76,6 +77,29 @@ class LateralViewJoinOperator extends NaryOperator[HiveLateralViewJoinOperator] udtfOp.initializeOnSlave() } + override def outputObjectInspector() = { + val SELECT_TAG = 0 + val UDTF_TAG = 1 + + val ois = new ArrayBuffer[ObjectInspector]() + val fieldNames = desc.getOutputInternalColNames() + + // The output of the lateral view join will be the columns from the select + // parent, followed by the column from the UDTF parent + var soi = objectInspectors(SELECT_TAG).asInstanceOf[StructObjectInspector] + + for (sf <- soi.getAllStructFieldRefs()) { + ois.add(sf.getFieldObjectInspector()); + } + + soi = objectInspectors(UDTF_TAG).asInstanceOf[StructObjectInspector] + for (sf <- soi.getAllStructFieldRefs()) { + ois.add(sf.getFieldObjectInspector()); + } + + ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, ois) + } + override def execute: RDD[_] = { // Execute LateralViewForwardOperator, bypassing Select / UDTF - Select // branches (see diagram in Hive's). diff --git a/src/main/scala/shark/execution/LimitOperator.scala b/src/main/scala/shark/execution/LimitOperator.scala index a66b0612..7788cbad 100755 --- a/src/main/scala/shark/execution/LimitOperator.scala +++ b/src/main/scala/shark/execution/LimitOperator.scala @@ -19,21 +19,22 @@ package shark.execution import scala.collection.Iterator import scala.reflect.BeanProperty - import org.apache.hadoop.hive.ql.exec.{LimitOperator => HiveLimitOperator} +import org.apache.hadoop.hive.ql.plan.LimitDesc import org.apache.spark.rdd.RDD import shark.SharkEnv -class LimitOperator extends UnaryOperator[HiveLimitOperator] { + +class LimitOperator extends UnaryOperator[LimitDesc] { // Only works on the master program. - def limit = hiveOp.getConf().getLimit() + def limit = desc.getLimit() override def execute(): RDD[_] = { - val limitNum = hiveOp.getConf().getLimit() + val limitNum = desc.getLimit() if (limitNum > 0) { // Take limit on each partition. diff --git a/src/main/scala/shark/execution/MapJoinOperator.scala b/src/main/scala/shark/execution/MapJoinOperator.scala index ea232b6e..d342c089 100755 --- a/src/main/scala/shark/execution/MapJoinOperator.scala +++ b/src/main/scala/shark/execution/MapJoinOperator.scala @@ -17,16 +17,20 @@ package shark.execution -import java.util.{HashMap => JHashMap, List => JList} +import java.util.{ArrayList, HashMap => JHashMap, List => JList} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.BeanProperty +import org.apache.hadoop.io.BytesWritable + import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, JoinUtil => HiveJoinUtil} import org.apache.hadoop.hive.ql.exec.{MapJoinOperator => HiveMapJoinOperator} import org.apache.hadoop.hive.ql.plan.MapJoinDesc import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.spark.rdd.RDD @@ -42,7 +46,7 @@ import shark.execution.serialization.{OperatorSerializationWrapper, Serializable * Different from Hive, we don't spill the hash tables to disk. If the "small" * tables are too big to fit in memory, the normal join should be used anyway. */ -class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperator] { +class MapJoinOperator extends CommonJoinOperator[MapJoinDesc] { @BeanProperty var posBigTable: Int = _ @BeanProperty var bigTableAlias: Int = _ @@ -79,6 +83,29 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc, HiveMapJoinOperato joinKeys, objectInspectors.toArray, CommonJoinOperator.NOTSKIPBIGTABLE) } + + // copied from the org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator + override def outputObjectInspector() = { + var outputObjInspector = super.outputObjectInspector() + var structFields = outputObjInspector.asInstanceOf[StructObjectInspector] + .getAllStructFieldRefs() + if (conf.getOutputColumnNames().size() < structFields.size()) { + var structFieldObjectInspectors = new ArrayList[ObjectInspector]() + for (alias <- order) { + var sz = conf.getExprs().get(alias).size() + var retained = conf.getRetainList().get(alias) + for (i <- 0 to sz - 1) { + var pos = retained.get(i) + structFieldObjectInspectors.add(structFields.get(pos).getFieldObjectInspector()) + } + } + outputObjInspector = ObjectInspectorFactory + .getStandardStructObjectInspector(conf.getOutputColumnNames(), + structFieldObjectInspectors) + } + + outputObjInspector + } override def execute(): RDD[_] = { val inputRdds = executeParents() diff --git a/src/main/scala/shark/execution/Operator.scala b/src/main/scala/shark/execution/Operator.scala index bee7a566..78aee5ff 100755 --- a/src/main/scala/shark/execution/Operator.scala +++ b/src/main/scala/shark/execution/Operator.scala @@ -17,13 +17,15 @@ package shark.execution -import java.util.{List => JavaList} - +import java.util.{List => JavaList, Arrays} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator import org.apache.spark.rdd.RDD @@ -31,7 +33,7 @@ import shark.LogHelper import shark.execution.serialization.OperatorSerializationWrapper -abstract class Operator[T <: HiveOperator] extends LogHelper with Serializable { +abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { /** * Initialize the operator on master node. This can have dependency on other @@ -60,7 +62,7 @@ abstract class Operator[T <: HiveOperator] extends LogHelper with Serializable { */ def initializeMasterOnAll() { _parentOperators.foreach(_.initializeMasterOnAll()) - objectInspectors ++= hiveOp.getInputObjInspectors() + objectInspectors = inputObjectInspectors() initializeOnMaster() } @@ -79,18 +81,18 @@ abstract class Operator[T <: HiveOperator] extends LogHelper with Serializable { * Return the parent operators as a Java List. This is for interoperability * with Java. We use this in explain's Java code. */ - def parentOperatorsAsJavaList: JavaList[Operator[_]] = _parentOperators + def parentOperatorsAsJavaList: JavaList[Operator[_<:HiveDesc]] = _parentOperators - def addParent(parent: Operator[_]) { + def addParent(parent: Operator[_<:HiveDesc]) { _parentOperators += parent parent.childOperators += this } - def addChild(child: Operator[_]) { + def addChild(child: Operator[_<:HiveDesc]) { child.addParent(this) } - def returnTerminalOperators(): Seq[Operator[_]] = { + def returnTerminalOperators(): Seq[Operator[_<:HiveDesc]] = { if (_childOperators == null || _childOperators.size == 0) { Seq(this) } else { @@ -106,14 +108,131 @@ abstract class Operator[T <: HiveOperator] extends LogHelper with Serializable { } } - @transient var hiveOp: T = _ - @transient private val _childOperators = new ArrayBuffer[Operator[_]]() - @transient private val _parentOperators = new ArrayBuffer[Operator[_]]() - @transient var objectInspectors = new ArrayBuffer[ObjectInspector] + def desc() = _desc + // TODO subclassing + def setDesc[B >: T](d: B) {_desc = d.asInstanceOf[T]} + + @transient private[this] var _desc: T = _ + @transient private[this] val _childOperators = new ArrayBuffer[Operator[_<:HiveDesc]]() + @transient private[this] val _parentOperators = new ArrayBuffer[Operator[_<:HiveDesc]]() + @transient var objectInspectors: Array[ObjectInspector] =_ protected def executeParents(): Seq[(Int, RDD[_])] = { parentOperators.map(p => (p.getTag, p.execute())) } + + protected def inputObjectInspectors(): Array[ObjectInspector] ={ + if(null != _parentOperators) + _parentOperators.sortBy(_.getTag).map(_.outputObjectInspector).toArray + else + null + } + + // derived classes can set this to different object if needed, default is the first input OI + def outputObjectInspector(): ObjectInspector = objectInspectors(0) + + /** + * Copy from the org.apache.hadoop.hive.ql.exec.ReduceSinkOperator + * Initializes array of ExprNodeEvaluator. Adds Union field for distinct + * column indices for group by. + * Puts the return values into a StructObjectInspector with output column + * names. + * + * If distinctColIndices is empty, the object inspector is same as + * {@link Operator#initEvaluatorsAndReturnStruct(ExprNodeEvaluator[], List, ObjectInspector)} + */ + protected def initEvaluatorsAndReturnStruct( + evals: Array[ExprNodeEvaluator] , distinctColIndices: JavaList[JavaList[Integer]] , + outputColNames: JavaList[String], length: Int, rowInspector: ObjectInspector): StructObjectInspector = { + + var fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); + initEvaluatorsAndReturnStruct(evals, fieldObjectInspectors, distinctColIndices, outputColNames, length, rowInspector) + } + + /** + * Copy from the org.apache.hadoop.hive.ql.exec.ReduceSinkOperator + * Initializes array of ExprNodeEvaluator. Adds Union field for distinct + * column indices for group by. + * Puts the return values into a StructObjectInspector with output column + * names. + * + * If distinctColIndices is empty, the object inspector is same as + * {@link Operator#initEvaluatorsAndReturnStruct(ExprNodeEvaluator[], List, ObjectInspector)} + */ + protected def initEvaluatorsAndReturnStruct( + evals: Array[ExprNodeEvaluator], fieldObjectInspectors: Array[ObjectInspector], distinctColIndices: JavaList[JavaList[Integer]] , + outputColNames: JavaList[String], length: Int, rowInspector: ObjectInspector): StructObjectInspector = { + + var inspectorLen = if (evals.length > length) length + 1 else evals.length + + var sois = new ArrayBuffer[ObjectInspector](inspectorLen) + + // keys + // var fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); + sois++=fieldObjectInspectors + + if (evals.length > length) { + // union keys + var uois = new ArrayBuffer[ObjectInspector]() + for (/*List*/ distinctCols <- distinctColIndices) { + var names = new ArrayBuffer[String]() + var eois = new ArrayBuffer[ObjectInspector]() + var numExprs = 0 + for (i <- distinctCols) { + names.add(HiveConf.getColumnInternalName(numExprs)) + eois.add(evals(i).initialize(rowInspector)) + numExprs += 1 + } + uois.add(ObjectInspectorFactory.getStandardStructObjectInspector(names, eois)); + } + + sois.add(ObjectInspectorFactory.getStandardUnionObjectInspector(uois)) + } + + ObjectInspectorFactory.getStandardStructObjectInspector(outputColNames, sois ) + } + + /** + * Initialize an array of ExprNodeEvaluator and return the result + * ObjectInspectors. + */ + protected def initEvaluators(evals: Array[ExprNodeEvaluator], + rowInspector: ObjectInspector): Array[ObjectInspector] = { + var result = new Array[ObjectInspector](evals.length) + for (i <- 0 to evals.length -1) { + result(i) = evals(i).initialize(rowInspector) + } + + result + } + + /** + * Initialize an array of ExprNodeEvaluator from start, for specified length + * and return the result ObjectInspectors. + */ + protected def initEvaluators(evals: Array[ExprNodeEvaluator], + start: Int, length: Int,rowInspector: ObjectInspector): Array[ObjectInspector] = { + var result = new Array[ObjectInspector](length) + + for (i <- 0 to length - 1) { + result(i) = evals(start + i).initialize(rowInspector); + } + + result + } + + /** + * Initialize an array of ExprNodeEvaluator and put the return values into a + * StructObjectInspector with integer field names. + */ + protected def initEvaluatorsAndReturnStruct( + evals: Array[ExprNodeEvaluator], outputColName: JavaList[String], + rowInspector: ObjectInspector): StructObjectInspector = { + import scala.collection.JavaConversions.JListWrapper + var fieldObjectInspectors = initEvaluators(evals, rowInspector) + return ObjectInspectorFactory.getStandardStructObjectInspector( + outputColName, fieldObjectInspectors.toList) + } } @@ -132,7 +251,7 @@ abstract class Operator[T <: HiveOperator] extends LogHelper with Serializable { * processPartition before sending it downstream. * */ -abstract class NaryOperator[T <: HiveOperator] extends Operator[T] { +abstract class NaryOperator[T <: HiveDesc] extends Operator[T] { /** Process a partition. Called on slaves. */ def processPartition(split: Int, iter: Iterator[_]): Iterator[_] @@ -168,7 +287,7 @@ abstract class NaryOperator[T <: HiveOperator] extends Operator[T] { * processPartition before sending it downstream. * */ -abstract class UnaryOperator[T <: HiveOperator] extends Operator[T] { +abstract class UnaryOperator[T <: HiveDesc] extends Operator[T] { /** Process a partition. Called on slaves. */ def processPartition(split: Int, iter: Iterator[_]): Iterator[_] @@ -192,7 +311,7 @@ abstract class UnaryOperator[T <: HiveOperator] extends Operator[T] { } -abstract class TopOperator[T <: HiveOperator] extends UnaryOperator[T] +abstract class TopOperator[T <: HiveDesc] extends UnaryOperator[T] object Operator extends LogHelper { @@ -205,7 +324,7 @@ object Operator extends LogHelper { * to do logging, but calling logging automatically adds a reference to the * operator (which is not serializable by Java) in the Spark closure. */ - def executeProcessPartition(operator: Operator[_ <: HiveOperator], rdd: RDD[_]): RDD[_] = { + def executeProcessPartition(operator: Operator[_ <: HiveDesc], rdd: RDD[_]): RDD[_] = { val op = OperatorSerializationWrapper(operator) rdd.mapPartitionsWithIndex { case(split, partition) => op.logDebug("Started executing mapPartitions for operator: " + op) diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 1bd1ca55..98cec40d 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -18,8 +18,9 @@ package shark.execution import scala.collection.JavaConversions._ - -import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator, GroupByPreShuffleOperator} +import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator} +import org.apache.hadoop.hive.ql.exec.GroupByPreShuffleOperator +import org.apache.hadoop.hive.ql.exec.{Operator => HOperator} import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.spark.storage.StorageLevel @@ -38,21 +39,24 @@ object OperatorFactory extends LogHelper { * uses Shark operators. This function automatically finds the Hive terminal * operator, and replicate the plan recursively up. */ - def createSharkPlan(hiveOp: HiveOperator): TerminalOperator = { + def createSharkPlan[T<:HiveDesc](hiveOp: HOperator[T]): TerminalOperator = { val hiveTerminalOp = _findHiveTerminalOperator(hiveOp) _createOperatorTree(hiveTerminalOp).asInstanceOf[TerminalOperator] } def createSharkMemoryStoreOutputPlan( - hiveTerminalOp: HiveOperator, + hiveTerminalOp: HOperator[_<:HiveDesc], tableName: String, storageLevel: StorageLevel, numColumns: Int, hivePartitionKey: String, cacheMode: CacheType, useUnionRDD: Boolean): TerminalOperator = { + // TODO the terminal operator is the FileSinkOperator in Hive? + var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] val sinkOp = _newOperatorInstance( - classOf[MemoryStoreSinkOperator], hiveTerminalOp).asInstanceOf[MemoryStoreSinkOperator] + classOf[MemoryStoreSinkOperator], hiveOp).asInstanceOf[MemoryStoreSinkOperator] + sinkOp.localHiveOp = hiveOp sinkOp.tableName = tableName sinkOp.storageLevel = storageLevel sinkOp.numColumns = numColumns @@ -62,74 +66,85 @@ object OperatorFactory extends LogHelper { _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } - def createSharkFileOutputPlan(hiveTerminalOp: HiveOperator): TerminalOperator = { - val sinkOp = _newOperatorInstance(classOf[FileSinkOperator], hiveTerminalOp) - _createAndSetParents( - sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] + def createSharkFileOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { + // TODO the terminal operator is the FileSinkOperator in Hive? + var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] + val sinkOp = _newOperatorInstance(classOf[FileSinkOperator], hiveOp).asInstanceOf[TerminalOperator] + sinkOp.localHiveOp = hiveOp + _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } - def createSharkRddOutputPlan(hiveTerminalOp: HiveOperator): TerminalOperator = { - val sinkOp = _newOperatorInstance(classOf[TableRddSinkOperator], hiveTerminalOp) + def createSharkRddOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { + // TODO the terminal operator is the FileSinkOperator in Hive? + var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] + val sinkOp = _newOperatorInstance(classOf[TableRddSinkOperator], hiveOp).asInstanceOf[TableRddSinkOperator] + sinkOp.localHiveOp = hiveOp _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } /** Create a Shark operator given the Hive operator. */ - private def createSingleOperator(hiveOp: HiveOperator): Operator[_] = { + private def createSingleOperator[T<:HiveDesc](hiveOp: HOperator[T]): Operator[T] = { // This is kind of annoying, but it works with strong typing ... val sharkOp = hiveOp match { case hop: org.apache.hadoop.hive.ql.exec.TableScanOperator => - _newOperatorInstance(classOf[TableScanOperator], hiveOp) + var op = _newOperatorInstance(classOf[TableScanOperator], hop) + op.asInstanceOf[TableScanOperator].hiveOp = hop + op case hop: org.apache.hadoop.hive.ql.exec.SelectOperator => - _newOperatorInstance(classOf[SelectOperator], hiveOp) + _newOperatorInstance(classOf[SelectOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.FileSinkOperator => - _newOperatorInstance(classOf[TerminalOperator], hiveOp) + var op = _newOperatorInstance(classOf[TerminalOperator], hop) + op.asInstanceOf[TerminalOperator].localHiveOp = hop + op case hop: org.apache.hadoop.hive.ql.exec.LimitOperator => - _newOperatorInstance(classOf[LimitOperator], hiveOp) + _newOperatorInstance(classOf[LimitOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.FilterOperator => - _newOperatorInstance(classOf[FilterOperator], hiveOp) + _newOperatorInstance(classOf[FilterOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.ReduceSinkOperator => - _newOperatorInstance(classOf[ReduceSinkOperator], hiveOp) + _newOperatorInstance(classOf[ReduceSinkOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.ExtractOperator => - _newOperatorInstance(classOf[ExtractOperator], hiveOp) + _newOperatorInstance(classOf[ExtractOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.UnionOperator => - _newOperatorInstance(classOf[UnionOperator], hiveOp) + _newOperatorInstance(classOf[UnionOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.JoinOperator => - _newOperatorInstance(classOf[JoinOperator], hiveOp) + _newOperatorInstance(classOf[JoinOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.MapJoinOperator => - _newOperatorInstance(classOf[MapJoinOperator], hiveOp) + _newOperatorInstance(classOf[MapJoinOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.ScriptOperator => - _newOperatorInstance(classOf[ScriptOperator], hiveOp) + var op = _newOperatorInstance(classOf[ScriptOperator], hop) + op.asInstanceOf[ScriptOperator].operatorId = hop.getOperatorId() + op case hop: org.apache.hadoop.hive.ql.exec.LateralViewForwardOperator => - _newOperatorInstance(classOf[LateralViewForwardOperator], hiveOp) + _newOperatorInstance(classOf[LateralViewForwardOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator => - _newOperatorInstance(classOf[LateralViewJoinOperator], hiveOp) + _newOperatorInstance(classOf[LateralViewJoinOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.UDTFOperator => - _newOperatorInstance(classOf[UDTFOperator], hiveOp) + _newOperatorInstance(classOf[UDTFOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.ForwardOperator => - _newOperatorInstance(classOf[ForwardOperator], hiveOp) + _newOperatorInstance(classOf[ForwardOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.GroupByOperator => { // For GroupBy, we separate post shuffle from pre shuffle. if (GroupByOperator.isPostShuffle(hop)) { - _newOperatorInstance(classOf[GroupByPostShuffleOperator], hiveOp) + _newOperatorInstance(classOf[GroupByPostShuffleOperator], hop) } else { - _newOperatorInstance(classOf[GroupByPreShuffleOperator], hiveOp) + _newOperatorInstance(classOf[GroupByPreShuffleOperator], hop) } } case _ => throw new HiveException("Unsupported Hive operator: " + hiveOp.getClass.getName) } logDebug("Replacing %s with %s".format(hiveOp.getClass.getName, sharkOp.getClass.getName)) - sharkOp + sharkOp.asInstanceOf[Operator[T]] } - private def _newOperatorInstance[T <: HiveOperator]( - cls: Class[_ <: Operator[T]], hiveOp: HiveOperator): Operator[_] = { + private def _newOperatorInstance[T <: HiveDesc]( + cls: Class[_ <: Operator[T]], hiveOp: HOperator[T]): Operator[T] = { val op = cls.newInstance() - op.hiveOp = hiveOp.asInstanceOf[T] + op.setDesc(hiveOp.getConf()) op } - private def _createAndSetParents(op: Operator[_], parents: Seq[HiveOperator]) = { + private def _createAndSetParents[T <: HiveDesc](op: Operator[T], parents: Seq[HOperator[_<:HiveDesc]]) = { if (parents != null) { parents foreach { parent => _createOperatorTree(parent).addChild(op) @@ -142,7 +157,7 @@ object OperatorFactory extends LogHelper { * Given a terminal operator in Hive, create the plan that uses Shark physical * operators. */ - private def _createOperatorTree(hiveOp: HiveOperator): Operator[_] = { + private def _createOperatorTree[T<:HiveDesc](hiveOp: HOperator[T]): Operator[T] = { val current = createSingleOperator(hiveOp) val parents = hiveOp.getParentOperators if (parents != null) { @@ -153,7 +168,7 @@ object OperatorFactory extends LogHelper { } } - private def _findHiveTerminalOperator(hiveOp: HiveOperator): HiveOperator = { + private def _findHiveTerminalOperator(hiveOp: HOperator[_<:HiveDesc]): HOperator[_<:HiveDesc] = { if (hiveOp.getChildOperators() == null || hiveOp.getChildOperators().size() == 0) { hiveOp } else { diff --git a/src/main/scala/shark/execution/ReduceSinkOperator.scala b/src/main/scala/shark/execution/ReduceSinkOperator.scala index 6c01ab47..2334ed24 100755 --- a/src/main/scala/shark/execution/ReduceSinkOperator.scala +++ b/src/main/scala/shark/execution/ReduceSinkOperator.scala @@ -23,6 +23,7 @@ import scala.collection.Iterator import scala.collection.JavaConversions._ import scala.reflect.BeanProperty +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{ReduceSinkOperator => HiveReduceSinkOperator} import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} import org.apache.hadoop.hive.ql.metadata.HiveException @@ -38,7 +39,7 @@ import org.apache.hadoop.io.BytesWritable * Converts a collection of rows into key, value pairs. This is the * upstream operator for joins and groupbys. */ -class ReduceSinkOperator extends UnaryOperator[HiveReduceSinkOperator] { +class ReduceSinkOperator extends UnaryOperator[ReduceSinkDesc] { @BeanProperty var conf: ReduceSinkDesc = _ @@ -58,16 +59,22 @@ class ReduceSinkOperator extends UnaryOperator[HiveReduceSinkOperator] { @transient var keySer: SerDe = _ @transient var valueSer: SerDe = _ @transient var keyObjInspector: ObjectInspector = _ + @transient var keyFieldObjInspectors: Array[ObjectInspector] = _ @transient var valObjInspector: ObjectInspector = _ + @transient var valFieldObjInspectors: Array[ObjectInspector] = _ @transient var partitionObjInspectors: Array[ObjectInspector] = _ override def getTag() = conf.getTag() override def initializeOnMaster() { - conf = hiveOp.getConf() + super.initializeOnMaster() + + conf = desc } override def initializeOnSlave() { + super.initializeOnSlave() + initializeOisAndSers(conf, objectInspector) } @@ -78,46 +85,28 @@ class ReduceSinkOperator extends UnaryOperator[HiveReduceSinkOperator] { processPartitionDistinct(iter) } } - - def initializeDownStreamHiveOperator() { - - conf = hiveOp.getConf() - - // Note that we get input object inspector from hiveOp rather than Shark's - // objectInspector because initializeMasterOnAll() hasn't been invoked yet. - initializeOisAndSers(conf, hiveOp.getInputObjInspectors().head) - - // Determine output object inspector (a struct of KEY, VALUE). + + override def outputObjectInspector() = { + initializeOisAndSers(conf, objectInspector) + val ois = new ArrayList[ObjectInspector] ois.add(keySer.getObjectInspector) ois.add(valueSer.getObjectInspector) - val outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector(List("KEY","VALUE"), ois) - - val joinTag = conf.getTag() - - // Propagate the output object inspector and serde infos to downstream operator. - childOperators.foreach { child => - child match { - case child: HiveTopOperator => { - child.setInputObjectInspector(joinTag, outputObjInspector) - child.setKeyValueTableDescs(joinTag, - (conf.getKeySerializeInfo, conf.getValueSerializeInfo)) - } - case _ => { - throw new HiveException("%s's downstream operator should be %s. %s found.".format( - this.getClass.getName, classOf[HiveTopOperator].getName, child.getClass.getName)) - } - } - } + ObjectInspectorFactory.getStandardStructObjectInspector(List("KEY","VALUE"), ois) } + // will be used of the children operators (in JoinOperator/Extractor/GroupByPostShuffleOperator + def getKeyValueTableDescs() = (conf.getKeySerializeInfo, conf.getValueSerializeInfo) + /** * Initialize the object inspectors, evaluators, and serializers. Used on * both the master and the slave. */ private def initializeOisAndSers(conf: ReduceSinkDesc, rowInspector: ObjectInspector) { keyEval = conf.getKeyCols.map(ExprNodeEvaluatorFactory.get(_)).toArray + keyFieldObjInspectors = initEvaluators(keyEval, 0, keyEval.length, rowInspector) + val numDistributionKeys = conf.getNumDistributionKeys() val distinctColIndices = conf.getDistinctColumnIndices() valueEval = conf.getValueCols.map(ExprNodeEvaluatorFactory.get(_)).toArray @@ -133,7 +122,7 @@ class ReduceSinkOperator extends UnaryOperator[HiveReduceSinkOperator] { valueSer.initialize(null, valueTableDesc.getProperties()) // Initialize object inspector for key columns. - keyObjInspector = ReduceSinkOperatorHelper.initEvaluatorsAndReturnStruct( + keyObjInspector = initEvaluatorsAndReturnStruct( keyEval, distinctColIndices, conf.getOutputKeyColumnNames, @@ -141,9 +130,9 @@ class ReduceSinkOperator extends UnaryOperator[HiveReduceSinkOperator] { rowInspector) // Initialize object inspector for value columns. - val valFieldInspectors = valueEval.map(eval => eval.initialize(rowInspector)).toList + valFieldObjInspectors = valueEval.map(eval => eval.initialize(rowInspector)) valObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector( - conf.getOutputValueColumnNames(), valFieldInspectors) + conf.getOutputValueColumnNames(), valFieldObjInspectors.toList) // Initialize evaluator and object inspector for partition columns. partitionEval = conf.getPartitionCols.map(ExprNodeEvaluatorFactory.get(_)).toArray diff --git a/src/main/scala/shark/execution/ReduceSinkTableDesc.scala b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala new file mode 100644 index 00000000..617c05ad --- /dev/null +++ b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution + +import shark.LogHelper +import org.apache.hadoop.hive.ql.plan.TableDesc + +trait ReduceSinkTableDesc extends LogHelper { + self: Operator[_ <: HiveDesc] => + + def keyValueDescs() = { + // get the parent ReduceSinkOperator and sort it by tag + var reduceSinkOps = for(op <- self.parentOperators; if(op.isInstanceOf[ReduceSinkOperator])) + yield op.asInstanceOf[ReduceSinkOperator] + + reduceSinkOps.map(f=>(f.getTag, f.getKeyValueTableDescs)) + } +} \ No newline at end of file diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 1881c8a3..33506ed6 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -48,12 +48,13 @@ import shark.LogHelper * * Example: select transform(key) using 'cat' as cola from src; */ -class ScriptOperator extends UnaryOperator[HiveScriptOperator] { +class ScriptOperator extends UnaryOperator[ScriptDesc] { - @BeanProperty var localHiveOp: HiveScriptOperator = _ @BeanProperty var localHconf: HiveConf = _ @BeanProperty var alias: String = _ + @BeanProperty var conf: ScriptDesc = _ + @transient var operatorId: String = _ @transient var scriptInputSerializer: Serializer = _ @transient var scriptOutputDeserializer: Deserializer = _ @@ -66,8 +67,8 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { val op = OperatorSerializationWrapper(this) val (command, envs) = getCommandAndEnvs() - val outRecordReaderClass: Class[_ <: RecordReader] = hiveOp.getConf().getOutRecordReaderClass() - val inRecordWriterClass: Class[_ <: RecordWriter] = hiveOp.getConf().getInRecordWriterClass() + val outRecordReaderClass: Class[_ <: RecordReader] = conf.getOutRecordReaderClass() + val inRecordWriterClass: Class[_ <: RecordWriter] = conf.getInRecordWriterClass() logDebug("Using %s and %s".format(outRecordReaderClass, inRecordWriterClass)) // Deserialize the output from script back to what Hive understands. @@ -144,31 +145,32 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { recordReader.initialize( proc.getInputStream, op.localHconf, - op.localHiveOp.getConf().getScriptOutputInfo().getProperties()) + op.conf.getScriptOutputInfo().getProperties()) op.deserializeFromScript(new ScriptOperator.RecordReaderIterator(recordReader)) } } override def initializeOnMaster() { - localHiveOp = hiveOp + super.initializeOnMaster() localHconf = super.hconf - // Set parent to null so we won't serialize the entire query plan. - hiveOp.setParentOperators(null) - hiveOp.setChildOperators(null) - hiveOp.setInputObjInspectors(null) + conf = desc + + initializeOnSlave() } + override def outputObjectInspector() = scriptOutputDeserializer.getObjectInspector() + override def initializeOnSlave() { - scriptOutputDeserializer = localHiveOp.getConf().getScriptOutputInfo() + scriptOutputDeserializer = conf.getScriptOutputInfo() .getDeserializerClass().newInstance() - scriptOutputDeserializer.initialize(localHconf, localHiveOp.getConf() + scriptOutputDeserializer.initialize(localHconf, conf .getScriptOutputInfo().getProperties()) - scriptInputSerializer = localHiveOp.getConf().getScriptInputInfo().getDeserializerClass() + scriptInputSerializer = conf.getScriptInputInfo().getDeserializerClass() .newInstance().asInstanceOf[Serializer] scriptInputSerializer.initialize( - localHconf, localHiveOp.getConf().getScriptInputInfo().getProperties()) + localHconf, conf.getScriptInputInfo().getProperties()) } /** @@ -180,7 +182,7 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { val scriptOpHelper = new ScriptOperatorHelper(new HiveScriptOperator) alias = scriptOpHelper.getAlias - val cmdArgs = HiveScriptOperator.splitArgs(hiveOp.getConf().getScriptCmd()) + val cmdArgs = HiveScriptOperator.splitArgs(conf.getScriptCmd()) val prog = cmdArgs(0) val currentDir = new File(".").getAbsoluteFile() @@ -209,7 +211,7 @@ class ScriptOperator extends UnaryOperator[HiveScriptOperator] { // Create an environment variable that uniquely identifies this script // operator val idEnvVarName = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVESCRIPTIDENVVAR) - val idEnvVarVal = hiveOp.getOperatorId() + val idEnvVarVal = operatorId envs.put(scriptOpHelper.safeEnvVarName(idEnvVarName), idEnvVarVal) (wrappedCmdArgs, Map.empty ++ envs) diff --git a/src/main/scala/shark/execution/SelectOperator.scala b/src/main/scala/shark/execution/SelectOperator.scala index 3361cfee..885c19ef 100755 --- a/src/main/scala/shark/execution/SelectOperator.scala +++ b/src/main/scala/shark/execution/SelectOperator.scala @@ -23,28 +23,36 @@ import scala.reflect.BeanProperty import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} import org.apache.hadoop.hive.ql.exec.{SelectOperator => HiveSelectOperator} import org.apache.hadoop.hive.ql.plan.SelectDesc +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector /** * An operator that does projection, i.e. selecting certain columns and * filtering out others. */ -class SelectOperator extends UnaryOperator[HiveSelectOperator] { +class SelectOperator extends UnaryOperator[SelectDesc] { @BeanProperty var conf: SelectDesc = _ @transient var evals: Array[ExprNodeEvaluator] = _ override def initializeOnMaster() { - conf = hiveOp.getConf() + super.initializeOnMaster() + conf = desc + initializeEvals(false) } - - override def initializeOnSlave() { + + def initializeEvals(initializeEval: Boolean) { if (!conf.isSelStarNoCompute) { evals = conf.getColList().map(ExprNodeEvaluatorFactory.get(_)).toArray - evals.foreach(_.initialize(objectInspector)) + if (initializeEval) { + evals.foreach(_.initialize(objectInspector)) + } } } + override def initializeOnSlave() { + initializeEvals(true) + } override def processPartition(split: Int, iter: Iterator[_]) = { if (conf.isSelStarNoCompute) { @@ -61,4 +69,11 @@ class SelectOperator extends UnaryOperator[HiveSelectOperator] { } } } + + override def outputObjectInspector(): ObjectInspector = { + if (conf.isSelStarNoCompute()) + super.outputObjectInspector() + else + initEvaluatorsAndReturnStruct(evals, conf.getOutputColumnNames(), objectInspector) + } } diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index 72bc1264..6aa9ffc7 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -86,9 +86,6 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { initializeTableScanTableDesc(tableScanOps) - // Initialize the Hive query plan. This gives us all the object inspectors. - initializeAllHiveOperators(terminalOp) - terminalOp.initializeMasterOnAll() // Set Spark's job description to be this query. @@ -140,28 +137,6 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { } } - def initializeAllHiveOperators(terminalOp: TerminalOperator) { - // Need to guarantee all parents are initialized before the child. - val topOpList = new scala.collection.mutable.MutableList[HiveTopOperator] - val queue = new scala.collection.mutable.Queue[Operator[_]] - queue.enqueue(terminalOp) - - while (!queue.isEmpty) { - val current = queue.dequeue() - current match { - case op: HiveTopOperator => topOpList += op - case _ => Unit - } - queue ++= current.parentOperators - } - - // Run the initialization. This guarantees that upstream operators are - // initialized before downstream ones. - topOpList.reverse.foreach { topOp => - topOp.initializeHiveTopOperator() - } - } - override def getType = StageType.MAPRED override def getName = "MAPRED-SPARK" diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 17816ca9..54a313f8 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -52,9 +52,10 @@ import shark.tachyon.TachyonException /** * The TableScanOperator is used for scanning any type of Shark or Hive table. */ -class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopOperator { +class TableScanOperator extends TopOperator[TableScanDesc] { @transient var table: Table = _ + @transient var hiveOp: HiveTableScanOperator = _ // Metadata for Hive-partitions (i.e if the table was created from PARTITION BY). NULL if this // table isn't Hive-partitioned. Set in SparkTask::initializeTableScanTableDesc(). @@ -74,52 +75,39 @@ class TableScanOperator extends TopOperator[HiveTableScanOperator] with HiveTopO @BeanProperty var tableDesc: TableDesc = _ - /** - * Initialize the hive TableScanOperator. This initialization propagates - * downstream. When all Hive TableScanOperators are initialized, the entire - * Hive query plan operators are initialized. - */ - override def initializeHiveTopOperator() { - - val rowObjectInspector = { - if (parts == null) { - val serializer = tableDesc.getDeserializerClass().newInstance() - serializer.initialize(hconf, tableDesc.getProperties) - serializer.getObjectInspector() - } else { - val partProps = firstConfPartDesc.getProperties() - val tableDeser = firstConfPartDesc.getDeserializerClass().newInstance() - tableDeser.initialize(hconf, partProps) - val partCols = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - val partNames = new ArrayList[String] - val partObjectInspectors = new ArrayList[ObjectInspector] - partCols.trim().split("/").foreach{ key => - partNames.add(key) - partObjectInspectors.add(PrimitiveObjectInspectorFactory.javaStringObjectInspector) - } - - // No need to lock this one (see SharkEnv.objectInspectorLock) because - // this is called on the master only. - val partObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( - partNames, partObjectInspectors) - val oiList = Arrays.asList( - tableDeser.getObjectInspector().asInstanceOf[StructObjectInspector], - partObjectInspector.asInstanceOf[StructObjectInspector]) - // new oi is union of table + partition object inspectors - ObjectInspectorFactory.getUnionStructObjectInspector(oiList) - } - } - - setInputObjectInspector(0, rowObjectInspector) - super.initializeHiveTopOperator() - } - override def initializeOnMaster() { // Create a local copy of the HiveConf that will be assigned job properties and, for disk reads, // broadcasted to slaves. localHConf = new HiveConf(super.hconf) } + override def outputObjectInspector() = { + if (parts == null) { + val serializer = tableDesc.getDeserializerClass().newInstance() + serializer.initialize(hconf, tableDesc.getProperties) + serializer.getObjectInspector() + } else { + val partProps = firstConfPartDesc.getProperties() + val tableDeser = firstConfPartDesc.getDeserializerClass().newInstance() + tableDeser.initialize(hconf, partProps) + val partCols = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + val partNames = new ArrayList[String] + val partObjectInspectors = new ArrayList[ObjectInspector] + partCols.trim().split("/").foreach{ key => + partNames.add(key) + partObjectInspectors.add(PrimitiveObjectInspectorFactory.javaStringObjectInspector) + } + + val partObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( + partNames, partObjectInspectors) + val oiList = Arrays.asList( + tableDeser.getObjectInspector().asInstanceOf[StructObjectInspector], + partObjectInspector.asInstanceOf[StructObjectInspector]) + // new oi is union of table + partition object inspectors + ObjectInspectorFactory.getUnionStructObjectInspector(oiList) + } + } + override def execute(): RDD[_] = { assert(parentOperators.size == 0) val tableKey: String = tableDesc.getTableName.split('.')(1) diff --git a/src/main/scala/shark/execution/TerminalOperator.scala b/src/main/scala/shark/execution/TerminalOperator.scala index 1a6400d7..7aa8afc8 100755 --- a/src/main/scala/shark/execution/TerminalOperator.scala +++ b/src/main/scala/shark/execution/TerminalOperator.scala @@ -23,6 +23,7 @@ import scala.reflect.BeanProperty import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} +import org.apache.hadoop.hive.ql.plan.FileSinkDesc /** @@ -31,7 +32,7 @@ import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} * - cache query output * - return query as RDD directly (without materializing it) */ -class TerminalOperator extends UnaryOperator[HiveFileSinkOperator] { +class TerminalOperator extends UnaryOperator[FileSinkDesc] { // Create a local copy of hconf and hiveSinkOp so we can XML serialize it. @BeanProperty var localHiveOp: HiveFileSinkOperator = _ @@ -39,12 +40,12 @@ class TerminalOperator extends UnaryOperator[HiveFileSinkOperator] { @BeanProperty val now = new Date() override def initializeOnMaster() { + super.initializeOnMaster() localHconf = super.hconf // Set parent to null so we won't serialize the entire query plan. - hiveOp.setParentOperators(null) - hiveOp.setChildOperators(null) - hiveOp.setInputObjInspectors(null) - localHiveOp = hiveOp + localHiveOp.setParentOperators(null) + localHiveOp.setChildOperators(null) + localHiveOp.setInputObjInspectors(null) } override def initializeOnSlave() { diff --git a/src/main/scala/shark/execution/UDTFOperator.scala b/src/main/scala/shark/execution/UDTFOperator.scala index db59f9cc..e680587b 100755 --- a/src/main/scala/shark/execution/UDTFOperator.scala +++ b/src/main/scala/shark/execution/UDTFOperator.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ ObjectInspector, StandardStructObjectInspector, StructField, StructObjectInspector } -class UDTFOperator extends UnaryOperator[HiveUDTFOperator] { +class UDTFOperator extends UnaryOperator[UDTFDesc] { @BeanProperty var conf: UDTFDesc = _ @@ -38,9 +38,14 @@ class UDTFOperator extends UnaryOperator[HiveUDTFOperator] { @transient var soi: StandardStructObjectInspector = _ @transient var inputFields: JavaList[_ <: StructField] = _ @transient var collector: UDTFCollector = _ + @transient var outputObjInspector: ObjectInspector = _ override def initializeOnMaster() { - conf = hiveOp.getConf() + super.initializeOnMaster() + + conf = desc + + initializeOnSlave() } override def initializeOnSlave() { @@ -56,9 +61,11 @@ class UDTFOperator extends UnaryOperator[HiveUDTFOperator] { }.toArray objToSendToUDTF = new Array[java.lang.Object](inputFields.size) - val udtfOutputOI = conf.getGenericUDTF().initialize(udtfInputOIs) + outputObjInspector = conf.getGenericUDTF().initialize(udtfInputOIs) } + override def outputObjectInspector() = outputObjInspector + override def processPartition(split: Int, iter: Iterator[_]): Iterator[_] = { iter.flatMap { row => explode(row) diff --git a/src/main/scala/shark/execution/UnionOperator.scala b/src/main/scala/shark/execution/UnionOperator.scala index 49157ef0..787733e1 100755 --- a/src/main/scala/shark/execution/UnionOperator.scala +++ b/src/main/scala/shark/execution/UnionOperator.scala @@ -25,12 +25,12 @@ import scala.reflect.BeanProperty import org.apache.hadoop.hive.ql.exec.{UnionOperator => HiveUnionOperator} import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ReturnObjectInspectorResolver +import org.apache.hadoop.hive.ql.plan.UnionDesc import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.StructField import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector - import org.apache.spark.rdd.{RDD, UnionRDD} import shark.execution.serialization.OperatorSerializationWrapper @@ -40,23 +40,28 @@ import shark.execution.serialization.OperatorSerializationWrapper * A union operator. If the incoming data are of different type, the union * operator transforms the incoming data into the same type. */ -class UnionOperator extends NaryOperator[HiveUnionOperator] { +class UnionOperator extends NaryOperator[UnionDesc] { - @transient var parentFields: ArrayBuffer[JavaList[_ <: StructField]] = _ - @transient var parentObjInspectors: ArrayBuffer[StructObjectInspector] = _ + @transient var parentFields: Array[JavaList[_ <: StructField]] = _ + @transient var parentObjInspectors: Array[StructObjectInspector] = _ @transient var columnTypeResolvers: Array[ReturnObjectInspectorResolver] = _ + @transient var outputObjInspector: ObjectInspector = _ @BeanProperty var needsTransform: Array[Boolean] = _ @BeanProperty var numParents: Int = _ override def initializeOnMaster() { + super.initializeOnMaster() numParents = parentOperators.size - // Use reflection to get the needsTransform boolean array. - val needsTransformField = hiveOp.getClass.getDeclaredField("needsTransform") - needsTransformField.setAccessible(true) - needsTransform = needsTransformField.get(hiveOp).asInstanceOf[Array[Boolean]] - + // whether we need to do transformation for each parent + var parents = parentOperators.length + var outputOI = outputObjectInspector() + needsTransform = Array.tabulate[Boolean](objectInspectors.length) { + // ObjectInspectors created by the ObjectInspectorFactory, which take the same ref if equals + i=>objectInspectors(i) != outputOI + } + initializeOnSlave() } @@ -82,7 +87,7 @@ class UnionOperator extends NaryOperator[HiveUnionOperator] { } val outputFieldOIs = columnTypeResolvers.map(_.get()) - val outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector( + outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector( columnNames, outputFieldOIs.toList) // whether we need to do transformation for each parent @@ -93,6 +98,8 @@ class UnionOperator extends NaryOperator[HiveUnionOperator] { p, objectInspectors(p), outputObjInspector)) } } + + override def outputObjectInspector() = outputObjInspector /** * Override execute. The only thing we need to call is combineMultipleRdds(). diff --git a/src/main/scala/shark/execution/package.scala b/src/main/scala/shark/execution/package.scala index e99b4766..eef52208 100755 --- a/src/main/scala/shark/execution/package.scala +++ b/src/main/scala/shark/execution/package.scala @@ -23,9 +23,9 @@ import shark.execution.serialization.OperatorSerializationWrapper package object execution { - type HiveOperator = org.apache.hadoop.hive.ql.exec.Operator[_] + type HiveDesc = java.io.Serializable // XXXDesc in Hive is the subclass of Serializable - implicit def opSerWrapper2op[T <: Operator[_ <: HiveOperator]]( + implicit def opSerWrapper2op[T <: Operator[_ <: HiveDesc]]( wrapper: OperatorSerializationWrapper[T]): T = wrapper.value implicit def kryoWrapper2object[T](wrapper: KryoSerializationWrapper[T]): T = wrapper.value diff --git a/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala b/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala index 858ce182..c0e026bf 100644 --- a/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala +++ b/src/main/scala/shark/execution/serialization/OperatorSerializationWrapper.scala @@ -17,7 +17,7 @@ package shark.execution.serialization -import shark.execution.HiveOperator +import shark.execution.HiveDesc import shark.execution.Operator @@ -28,7 +28,7 @@ import shark.execution.Operator * * Use OperatorSerializationWrapper(operator) to create a wrapper. */ -class OperatorSerializationWrapper[T <: Operator[_ <: HiveOperator]] +class OperatorSerializationWrapper[T <: Operator[_ <: HiveDesc]] extends Serializable with shark.LogHelper { /** The operator we are going to serialize. */ @@ -69,7 +69,7 @@ class OperatorSerializationWrapper[T <: Operator[_ <: HiveOperator]] object OperatorSerializationWrapper { - def apply[T <: Operator[_ <: HiveOperator]](value: T): OperatorSerializationWrapper[T] = { + def apply[T <: Operator[_ <: HiveDesc]](value: T): OperatorSerializationWrapper[T] = { val wrapper = new OperatorSerializationWrapper[T] wrapper.value = value wrapper diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index dea392d9..d3181859 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} import org.apache.hadoop.hive.metastore.Warehouse import org.apache.hadoop.hive.ql.exec.{DDLTask, FetchTask, MoveTask, TaskFactory} -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator, Operator => HiveOperator} import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.optimizer.Optimizer import org.apache.hadoop.hive.ql.parse._ @@ -297,7 +297,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // The move task depends on all root tasks. In the case of multi outputs, // the moves are only started once all outputs are executed. - val hiveFileSinkOp = terminalOps.head.hiveOp + val hiveFileSinkOp = terminalOps.head.localHiveOp mvTasks.foreach { moveTask => rootTasks.foreach { rootTask => rootTask.addDependentTask(moveTask) @@ -454,9 +454,9 @@ object SharkSemanticAnalyzer extends LogHelper { * Given a Hive top operator (e.g. TableScanOperator), find all the file sink * operators (aka file output operator). */ - private def findAllHiveFileSinkOperators(op: HiveOperator): Seq[HiveOperator] = { + private def findAllHiveFileSinkOperators(op: HiveOperator[_<: HiveDesc]): Seq[HiveOperator[_<: HiveDesc]] = { if (op.getChildOperators() == null || op.getChildOperators().size() == 0) { - Seq[HiveOperator](op) + Seq[HiveOperator[_<: HiveDesc]](op) } else { op.getChildOperators().flatMap(findAllHiveFileSinkOperators(_)).distinct } @@ -471,7 +471,7 @@ object SharkSemanticAnalyzer extends LogHelper { */ private def breakHivePlanByStages(terminalOps: Seq[TerminalOperator]) = { val reduceSinks = new scala.collection.mutable.HashSet[ReduceSinkOperator] - val queue = new scala.collection.mutable.Queue[Operator[_]] + val queue = new scala.collection.mutable.Queue[Operator[_ <: HiveDesc]] queue ++= terminalOps while (!queue.isEmpty) { @@ -486,15 +486,5 @@ object SharkSemanticAnalyzer extends LogHelper { } logDebug("Found %d ReduceSinkOperator's.".format(reduceSinks.size)) - - reduceSinks.foreach { op => - val hiveOp = op.asInstanceOf[Operator[HiveOperator]].hiveOp - if (hiveOp.getChildOperators() != null) { - hiveOp.getChildOperators().foreach { child => - logDebug("Removing child %s from %s".format(child, hiveOp)) - hiveOp.removeChild(child) - } - } - } } } From 8b35aefa6fc8770e1305eb74e52e964640463c0e Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sun, 29 Sep 2013 13:52:04 +0800 Subject: [PATCH 175/331] Update code according scala code style Conflicts: src/main/scala/shark/parse/SharkSemanticAnalyzer.scala --- .../shark/execution/ExtractOperator.scala | 4 +++- .../GroupByPostShuffleOperator.scala | 5 +++- .../execution/LateralViewJoinOperator.scala | 7 ++++-- .../scala/shark/execution/LimitOperator.scala | 1 + src/main/scala/shark/execution/Operator.scala | 23 +++++++++++-------- .../shark/execution/OperatorFactory.scala | 10 +++++--- .../shark/execution/ReduceSinkTableDesc.scala | 8 ++++--- .../scala/shark/execution/UnionOperator.scala | 5 ++-- .../shark/parse/SharkSemanticAnalyzer.scala | 6 +++-- 9 files changed, 45 insertions(+), 24 deletions(-) diff --git a/src/main/scala/shark/execution/ExtractOperator.scala b/src/main/scala/shark/execution/ExtractOperator.scala index 6f7c3467..2d74c71c 100755 --- a/src/main/scala/shark/execution/ExtractOperator.scala +++ b/src/main/scala/shark/execution/ExtractOperator.scala @@ -18,15 +18,17 @@ package shark.execution import scala.reflect.BeanProperty + import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} import org.apache.hadoop.hive.ql.exec.{ExtractOperator => HiveExtractOperator} import org.apache.hadoop.hive.ql.plan.{ExtractDesc, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.io.BytesWritable import org.apache.spark.rdd.RDD + import shark.SharkConfVars -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector class ExtractOperator extends UnaryOperator[ExtractDesc] diff --git a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala index 94748161..df00112b 100755 --- a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala @@ -19,13 +19,16 @@ package org.apache.hadoop.hive.ql.exec // Put this file in Hive's exec package to access package level visible fields and methods. import java.util.{ArrayList => JArrayList, HashMap => JHashMap, HashSet => JHashSet, Set => JSet} + import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.BeanProperty + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.plan.{ExprNodeColumnDesc, TableDesc} import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorUtils,StandardStructObjectInspector, StructObjectInspector, UnionObject} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorUtils, + StandardStructObjectInspector, StructObjectInspector, UnionObject} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils diff --git a/src/main/scala/shark/execution/LateralViewJoinOperator.scala b/src/main/scala/shark/execution/LateralViewJoinOperator.scala index cba93e43..2f2b5e39 100755 --- a/src/main/scala/shark/execution/LateralViewJoinOperator.scala +++ b/src/main/scala/shark/execution/LateralViewJoinOperator.scala @@ -19,18 +19,21 @@ package shark.execution import java.nio.ByteBuffer import java.util.ArrayList + +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.BeanProperty + import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} import org.apache.hadoop.hive.ql.exec.{LateralViewJoinOperator => HiveLateralViewJoinOperator} import org.apache.hadoop.hive.ql.plan.SelectDesc import org.apache.hadoop.hive.ql.plan.LateralViewJoinDesc +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} + import org.apache.spark.rdd.RDD import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} -import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory /** diff --git a/src/main/scala/shark/execution/LimitOperator.scala b/src/main/scala/shark/execution/LimitOperator.scala index 7788cbad..2c76fca9 100755 --- a/src/main/scala/shark/execution/LimitOperator.scala +++ b/src/main/scala/shark/execution/LimitOperator.scala @@ -19,6 +19,7 @@ package shark.execution import scala.collection.Iterator import scala.reflect.BeanProperty + import org.apache.hadoop.hive.ql.exec.{LimitOperator => HiveLimitOperator} import org.apache.hadoop.hive.ql.plan.LimitDesc diff --git a/src/main/scala/shark/execution/Operator.scala b/src/main/scala/shark/execution/Operator.scala index 78aee5ff..e634e025 100755 --- a/src/main/scala/shark/execution/Operator.scala +++ b/src/main/scala/shark/execution/Operator.scala @@ -113,19 +113,19 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { def setDesc[B >: T](d: B) {_desc = d.asInstanceOf[T]} @transient private[this] var _desc: T = _ - @transient private[this] val _childOperators = new ArrayBuffer[Operator[_<:HiveDesc]]() - @transient private[this] val _parentOperators = new ArrayBuffer[Operator[_<:HiveDesc]]() - @transient var objectInspectors: Array[ObjectInspector] =_ + @transient private val _childOperators = new ArrayBuffer[Operator[_<:HiveDesc]]() + @transient private val _parentOperators = new ArrayBuffer[Operator[_<:HiveDesc]]() + @transient var objectInspectors: Seq[ObjectInspector] =_ protected def executeParents(): Seq[(Int, RDD[_])] = { parentOperators.map(p => (p.getTag, p.execute())) } - protected def inputObjectInspectors(): Array[ObjectInspector] ={ + protected def inputObjectInspectors(): Seq[ObjectInspector] = { if(null != _parentOperators) - _parentOperators.sortBy(_.getTag).map(_.outputObjectInspector).toArray + _parentOperators.sortBy(_.getTag).map(_.outputObjectInspector) else - null + Seq.empty[ObjectInspector] } // derived classes can set this to different object if needed, default is the first input OI @@ -143,10 +143,12 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { */ protected def initEvaluatorsAndReturnStruct( evals: Array[ExprNodeEvaluator] , distinctColIndices: JavaList[JavaList[Integer]] , - outputColNames: JavaList[String], length: Int, rowInspector: ObjectInspector): StructObjectInspector = { + outputColNames: JavaList[String], length: Int, rowInspector: ObjectInspector): + StructObjectInspector = { var fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); - initEvaluatorsAndReturnStruct(evals, fieldObjectInspectors, distinctColIndices, outputColNames, length, rowInspector) + initEvaluatorsAndReturnStruct(evals, fieldObjectInspectors, distinctColIndices, + outputColNames, length, rowInspector) } /** @@ -160,8 +162,9 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { * {@link Operator#initEvaluatorsAndReturnStruct(ExprNodeEvaluator[], List, ObjectInspector)} */ protected def initEvaluatorsAndReturnStruct( - evals: Array[ExprNodeEvaluator], fieldObjectInspectors: Array[ObjectInspector], distinctColIndices: JavaList[JavaList[Integer]] , - outputColNames: JavaList[String], length: Int, rowInspector: ObjectInspector): StructObjectInspector = { + evals: Array[ExprNodeEvaluator], fieldObjectInspectors: Array[ObjectInspector], + distinctColIndices: JavaList[JavaList[Integer]], outputColNames: JavaList[String], + length: Int, rowInspector: ObjectInspector): StructObjectInspector = { var inspectorLen = if (evals.length > length) length + 1 else evals.length diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 98cec40d..ec0e1647 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -18,6 +18,7 @@ package shark.execution import scala.collection.JavaConversions._ + import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator} import org.apache.hadoop.hive.ql.exec.GroupByPreShuffleOperator import org.apache.hadoop.hive.ql.exec.{Operator => HOperator} @@ -69,7 +70,8 @@ object OperatorFactory extends LogHelper { def createSharkFileOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] - val sinkOp = _newOperatorInstance(classOf[FileSinkOperator], hiveOp).asInstanceOf[TerminalOperator] + val sinkOp = _newOperatorInstance(classOf[FileSinkOperator], + hiveOp).asInstanceOf[TerminalOperator] sinkOp.localHiveOp = hiveOp _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } @@ -77,7 +79,8 @@ object OperatorFactory extends LogHelper { def createSharkRddOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] - val sinkOp = _newOperatorInstance(classOf[TableRddSinkOperator], hiveOp).asInstanceOf[TableRddSinkOperator] + val sinkOp = _newOperatorInstance(classOf[TableRddSinkOperator], + hiveOp).asInstanceOf[TableRddSinkOperator] sinkOp.localHiveOp = hiveOp _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } @@ -144,7 +147,8 @@ object OperatorFactory extends LogHelper { op } - private def _createAndSetParents[T <: HiveDesc](op: Operator[T], parents: Seq[HOperator[_<:HiveDesc]]) = { + private def _createAndSetParents[T <: HiveDesc](op: Operator[T], + parents: Seq[HOperator[_<:HiveDesc]]) = { if (parents != null) { parents foreach { parent => _createOperatorTree(parent).addChild(op) diff --git a/src/main/scala/shark/execution/ReduceSinkTableDesc.scala b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala index 617c05ad..5447b89d 100644 --- a/src/main/scala/shark/execution/ReduceSinkTableDesc.scala +++ b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala @@ -23,10 +23,12 @@ import org.apache.hadoop.hive.ql.plan.TableDesc trait ReduceSinkTableDesc extends LogHelper { self: Operator[_ <: HiveDesc] => - def keyValueDescs() = { + // Seq(tag, (Key TableDesc, Value TableDesc)) + def keyValueDescs(): Seq[(Int, (TableDesc, TableDesc))] = { // get the parent ReduceSinkOperator and sort it by tag - var reduceSinkOps = for(op <- self.parentOperators; if(op.isInstanceOf[ReduceSinkOperator])) - yield op.asInstanceOf[ReduceSinkOperator] + var reduceSinkOps = for(op <- self.parentOperators.toSeq; + if(op.isInstanceOf[ReduceSinkOperator])) + yield op.asInstanceOf[ReduceSinkOperator] reduceSinkOps.map(f=>(f.getTag, f.getKeyValueTableDescs)) } diff --git a/src/main/scala/shark/execution/UnionOperator.scala b/src/main/scala/shark/execution/UnionOperator.scala index 787733e1..0e2a2270 100755 --- a/src/main/scala/shark/execution/UnionOperator.scala +++ b/src/main/scala/shark/execution/UnionOperator.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.StructField import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector + import org.apache.spark.rdd.{RDD, UnionRDD} import shark.execution.serialization.OperatorSerializationWrapper @@ -42,8 +43,8 @@ import shark.execution.serialization.OperatorSerializationWrapper */ class UnionOperator extends NaryOperator[UnionDesc] { - @transient var parentFields: Array[JavaList[_ <: StructField]] = _ - @transient var parentObjInspectors: Array[StructObjectInspector] = _ + @transient var parentFields: Seq[JavaList[_ <: StructField]] = _ + @transient var parentObjInspectors: Seq[StructObjectInspector] = _ @transient var columnTypeResolvers: Array[ReturnObjectInspectorResolver] = _ @transient var outputObjInspector: ObjectInspector = _ diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index d3181859..6bb6b74e 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -29,7 +29,8 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} import org.apache.hadoop.hive.metastore.Warehouse import org.apache.hadoop.hive.ql.exec.{DDLTask, FetchTask, MoveTask, TaskFactory} -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator, Operator => HiveOperator} +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator, + Operator => HiveOperator} import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.optimizer.Optimizer import org.apache.hadoop.hive.ql.parse._ @@ -454,7 +455,8 @@ object SharkSemanticAnalyzer extends LogHelper { * Given a Hive top operator (e.g. TableScanOperator), find all the file sink * operators (aka file output operator). */ - private def findAllHiveFileSinkOperators(op: HiveOperator[_<: HiveDesc]): Seq[HiveOperator[_<: HiveDesc]] = { + private def findAllHiveFileSinkOperators(op: HiveOperator[_<: HiveDesc]): + Seq[HiveOperator[_<: HiveDesc]] = { if (op.getChildOperators() == null || op.getChildOperators().size() == 0) { Seq[HiveOperator[_<: HiveDesc]](op) } else { From 081b30097a81737b826f3a2e7f997de11506caf5 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 15 Oct 2013 16:35:35 +0800 Subject: [PATCH 176/331] Update code style --- .../shark/execution/CommonJoinOperator.scala | 6 ++-- .../shark/execution/ExtractOperator.scala | 1 + .../execution/GroupByPreShuffleOperator.scala | 20 +++++------ src/main/scala/shark/execution/JoinUtil.scala | 5 +-- .../shark/execution/MapJoinOperator.scala | 7 ++-- src/main/scala/shark/execution/Operator.scala | 36 +++++++++---------- .../shark/execution/OperatorFactory.scala | 4 +-- .../shark/execution/ReduceSinkOperator.scala | 14 ++++---- .../shark/execution/ReduceSinkTableDesc.scala | 14 ++++---- .../shark/execution/ScriptOperator.scala | 13 +++---- .../shark/execution/SelectOperator.scala | 6 ++-- .../shark/execution/TableScanOperator.scala | 8 ++--- .../scala/shark/execution/UDTFOperator.scala | 2 +- .../scala/shark/execution/UnionOperator.scala | 13 +++---- .../shark/parse/SharkSemanticAnalyzer.scala | 4 +-- 15 files changed, 78 insertions(+), 75 deletions(-) diff --git a/src/main/scala/shark/execution/CommonJoinOperator.scala b/src/main/scala/shark/execution/CommonJoinOperator.scala index 5e7ce572..6c84044e 100755 --- a/src/main/scala/shark/execution/CommonJoinOperator.scala +++ b/src/main/scala/shark/execution/CommonJoinOperator.scala @@ -104,8 +104,8 @@ abstract class CommonJoinOperator[T <: JoinDesc] extends NaryOperator[T] { } ObjectInspectorFactory.getStandardStructObjectInspector( - conf.getOutputColumnNames(), - structFieldObjectInspectors) + conf.getOutputColumnNames(), + structFieldObjectInspectors) } } @@ -219,7 +219,7 @@ object CommonJoinOperator { def isFiltered(row: Any, filters: JavaList[ExprNodeEvaluator], ois: JavaList[ObjectInspector]) : Boolean = { // if no filter, then will not be filtered - if(filters == null || ois == null) return false + if (filters == null || ois == null) return false var ret: java.lang.Boolean = false var j = 0 diff --git a/src/main/scala/shark/execution/ExtractOperator.scala b/src/main/scala/shark/execution/ExtractOperator.scala index 2d74c71c..fe175d1b 100755 --- a/src/main/scala/shark/execution/ExtractOperator.scala +++ b/src/main/scala/shark/execution/ExtractOperator.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.{ExtractOperator => HiveExtractOperator} import org.apache.hadoop.hive.ql.plan.{ExtractDesc, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.io.BytesWritable + import org.apache.spark.rdd.RDD import shark.SharkConfVars diff --git a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala index c2275f59..afc86b53 100755 --- a/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPreShuffleOperator.scala @@ -50,6 +50,7 @@ class GroupByPreShuffleOperator extends UnaryOperator[GroupByDesc] { // The aggregation functions. @transient var aggregationEvals: Array[GenericUDAFEvaluator] = _ @transient var aggregationObjectInspectors: Array[ObjectInspector] = _ + // Key fields to be grouped. @transient var keyFields: Array[ExprNodeEvaluator] = _ // A struct object inspector composing of all the fields. @@ -68,10 +69,9 @@ class GroupByPreShuffleOperator extends UnaryOperator[GroupByDesc] { rowInspector = objectInspector.asInstanceOf[StructObjectInspector] keyFields = conf.getKeys().map(k => ExprNodeEvaluatorFactory.get(k)).toArray val keyObjectInspectors: Array[ObjectInspector] = keyFields.map(k => k.initialize(rowInspector)) - currentKeyObjectInspectors = - keyObjectInspectors.map { k => - ObjectInspectorUtils.getStandardObjectInspector(k, ObjectInspectorCopyOption.WRITABLE) - } + currentKeyObjectInspectors = keyObjectInspectors.map { k => + ObjectInspectorUtils.getStandardObjectInspector(k, ObjectInspectorCopyOption.WRITABLE) + } aggregationParameterFields = conf.getAggregators.toArray.map { aggr => aggr.asInstanceOf[AggregationDesc].getParameters.toArray.map { param => @@ -94,7 +94,7 @@ class GroupByPreShuffleOperator extends UnaryOperator[GroupByDesc] { aggregationObjectInspectors = Array.tabulate[ObjectInspector](aggregationEvals.length) { i=> - var mode = conf.getAggregators()(i).getMode() + val mode = conf.getAggregators()(i).getMode() aggregationEvals(i).init(mode, aggregationParameterObjectInspectors(i)) } @@ -128,13 +128,13 @@ class GroupByPreShuffleOperator extends UnaryOperator[GroupByDesc] { // copied from the org.apache.hadoop.hive.ql.exec.GroupByOperator override def outputObjectInspector() = { - var totalFields = keyFields.length + aggregationEvals.length + val totalFields = keyFields.length + aggregationEvals.length - var ois = new ArrayBuffer[ObjectInspector](totalFields) - ois.++=(currentKeyObjectInspectors) - ois.++=(aggregationObjectInspectors) + val ois = new ArrayBuffer[ObjectInspector](totalFields) + ois ++= (currentKeyObjectInspectors) + ois ++= (aggregationObjectInspectors) - var fieldNames = conf.getOutputColumnNames() + val fieldNames = conf.getOutputColumnNames() import scala.collection.JavaConversions._ ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, ois.toList) diff --git a/src/main/scala/shark/execution/JoinUtil.scala b/src/main/scala/shark/execution/JoinUtil.scala index 3fdf1806..9c6590c7 100644 --- a/src/main/scala/shark/execution/JoinUtil.scala +++ b/src/main/scala/shark/execution/JoinUtil.scala @@ -56,9 +56,9 @@ object JoinUtil { noOuterJoin: Boolean): Array[AnyRef] = { val isFiltered: Boolean = { - if(filters == null) + if (filters == null) { false - else + } else { Range(0, filters.size()).exists { x => val cond = filters.get(x).evaluate(row) val result = Option[AnyRef]( @@ -68,6 +68,7 @@ object JoinUtil { case None => true } } + } } val size = valueFields.size val a = new Array[AnyRef](size) diff --git a/src/main/scala/shark/execution/MapJoinOperator.scala b/src/main/scala/shark/execution/MapJoinOperator.scala index d342c089..87644387 100755 --- a/src/main/scala/shark/execution/MapJoinOperator.scala +++ b/src/main/scala/shark/execution/MapJoinOperator.scala @@ -87,8 +87,8 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc] { // copied from the org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator override def outputObjectInspector() = { var outputObjInspector = super.outputObjectInspector() - var structFields = outputObjInspector.asInstanceOf[StructObjectInspector] - .getAllStructFieldRefs() + val structFields = outputObjInspector.asInstanceOf[StructObjectInspector] + .getAllStructFieldRefs() if (conf.getOutputColumnNames().size() < structFields.size()) { var structFieldObjectInspectors = new ArrayList[ObjectInspector]() for (alias <- order) { @@ -100,7 +100,8 @@ class MapJoinOperator extends CommonJoinOperator[MapJoinDesc] { } } outputObjInspector = ObjectInspectorFactory - .getStandardStructObjectInspector(conf.getOutputColumnNames(), + .getStandardStructObjectInspector( + conf.getOutputColumnNames(), structFieldObjectInspectors) } diff --git a/src/main/scala/shark/execution/Operator.scala b/src/main/scala/shark/execution/Operator.scala index e634e025..7adaed34 100755 --- a/src/main/scala/shark/execution/Operator.scala +++ b/src/main/scala/shark/execution/Operator.scala @@ -109,7 +109,7 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { } def desc() = _desc - // TODO subclassing + def setDesc[B >: T](d: B) {_desc = d.asInstanceOf[T]} @transient private[this] var _desc: T = _ @@ -122,10 +122,11 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { } protected def inputObjectInspectors(): Seq[ObjectInspector] = { - if(null != _parentOperators) + if (null != _parentOperators) { _parentOperators.sortBy(_.getTag).map(_.outputObjectInspector) - else + } else { Seq.empty[ObjectInspector] + } } // derived classes can set this to different object if needed, default is the first input OI @@ -146,9 +147,9 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { outputColNames: JavaList[String], length: Int, rowInspector: ObjectInspector): StructObjectInspector = { - var fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); + val fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); initEvaluatorsAndReturnStruct(evals, fieldObjectInspectors, distinctColIndices, - outputColNames, length, rowInspector) + outputColNames, length, rowInspector) } /** @@ -166,33 +167,33 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { distinctColIndices: JavaList[JavaList[Integer]], outputColNames: JavaList[String], length: Int, rowInspector: ObjectInspector): StructObjectInspector = { - var inspectorLen = if (evals.length > length) length + 1 else evals.length + val inspectorLen = if (evals.length > length) length + 1 else evals.length - var sois = new ArrayBuffer[ObjectInspector](inspectorLen) + val sois = new ArrayBuffer[ObjectInspector](inspectorLen) // keys // var fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); - sois++=fieldObjectInspectors + sois ++= fieldObjectInspectors if (evals.length > length) { // union keys - var uois = new ArrayBuffer[ObjectInspector]() + val uois = new ArrayBuffer[ObjectInspector]() for (/*List*/ distinctCols <- distinctColIndices) { - var names = new ArrayBuffer[String]() - var eois = new ArrayBuffer[ObjectInspector]() + val names = new ArrayBuffer[String]() + val eois = new ArrayBuffer[ObjectInspector]() var numExprs = 0 for (i <- distinctCols) { names.add(HiveConf.getColumnInternalName(numExprs)) eois.add(evals(i).initialize(rowInspector)) numExprs += 1 } - uois.add(ObjectInspectorFactory.getStandardStructObjectInspector(names, eois)); + uois.add(ObjectInspectorFactory.getStandardStructObjectInspector(names, eois)) } sois.add(ObjectInspectorFactory.getStandardUnionObjectInspector(uois)) } - ObjectInspectorFactory.getStandardStructObjectInspector(outputColNames, sois ) + ObjectInspectorFactory.getStandardStructObjectInspector(outputColNames, sois) } /** @@ -201,7 +202,7 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { */ protected def initEvaluators(evals: Array[ExprNodeEvaluator], rowInspector: ObjectInspector): Array[ObjectInspector] = { - var result = new Array[ObjectInspector](evals.length) + val result = new Array[ObjectInspector](evals.length) for (i <- 0 to evals.length -1) { result(i) = evals(i).initialize(rowInspector) } @@ -215,7 +216,7 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { */ protected def initEvaluators(evals: Array[ExprNodeEvaluator], start: Int, length: Int,rowInspector: ObjectInspector): Array[ObjectInspector] = { - var result = new Array[ObjectInspector](length) + val result = new Array[ObjectInspector](length) for (i <- 0 to length - 1) { result(i) = evals(start + i).initialize(rowInspector); @@ -231,10 +232,9 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { protected def initEvaluatorsAndReturnStruct( evals: Array[ExprNodeEvaluator], outputColName: JavaList[String], rowInspector: ObjectInspector): StructObjectInspector = { - import scala.collection.JavaConversions.JListWrapper - var fieldObjectInspectors = initEvaluators(evals, rowInspector) + val fieldObjectInspectors = initEvaluators(evals, rowInspector) return ObjectInspectorFactory.getStandardStructObjectInspector( - outputColName, fieldObjectInspectors.toList) + outputColName, fieldObjectInspectors.toList) } } diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index ec0e1647..6992cdea 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -71,7 +71,7 @@ object OperatorFactory extends LogHelper { // TODO the terminal operator is the FileSinkOperator in Hive? var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] val sinkOp = _newOperatorInstance(classOf[FileSinkOperator], - hiveOp).asInstanceOf[TerminalOperator] + hiveOp).asInstanceOf[TerminalOperator] sinkOp.localHiveOp = hiveOp _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } @@ -80,7 +80,7 @@ object OperatorFactory extends LogHelper { // TODO the terminal operator is the FileSinkOperator in Hive? var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] val sinkOp = _newOperatorInstance(classOf[TableRddSinkOperator], - hiveOp).asInstanceOf[TableRddSinkOperator] + hiveOp).asInstanceOf[TableRddSinkOperator] sinkOp.localHiveOp = hiveOp _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } diff --git a/src/main/scala/shark/execution/ReduceSinkOperator.scala b/src/main/scala/shark/execution/ReduceSinkOperator.scala index 2334ed24..8dfd5a8a 100755 --- a/src/main/scala/shark/execution/ReduceSinkOperator.scala +++ b/src/main/scala/shark/execution/ReduceSinkOperator.scala @@ -85,7 +85,7 @@ class ReduceSinkOperator extends UnaryOperator[ReduceSinkDesc] { processPartitionDistinct(iter) } } - + override def outputObjectInspector() = { initializeOisAndSers(conf, objectInspector) @@ -123,16 +123,16 @@ class ReduceSinkOperator extends UnaryOperator[ReduceSinkDesc] { // Initialize object inspector for key columns. keyObjInspector = initEvaluatorsAndReturnStruct( - keyEval, - distinctColIndices, - conf.getOutputKeyColumnNames, - numDistributionKeys, - rowInspector) + keyEval, + distinctColIndices, + conf.getOutputKeyColumnNames, + numDistributionKeys, + rowInspector) // Initialize object inspector for value columns. valFieldObjInspectors = valueEval.map(eval => eval.initialize(rowInspector)) valObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector( - conf.getOutputValueColumnNames(), valFieldObjInspectors.toList) + conf.getOutputValueColumnNames(), valFieldObjInspectors.toList) // Initialize evaluator and object inspector for partition columns. partitionEval = conf.getPartitionCols.map(ExprNodeEvaluatorFactory.get(_)).toArray diff --git a/src/main/scala/shark/execution/ReduceSinkTableDesc.scala b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala index 5447b89d..c7952263 100644 --- a/src/main/scala/shark/execution/ReduceSinkTableDesc.scala +++ b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala @@ -22,14 +22,14 @@ import org.apache.hadoop.hive.ql.plan.TableDesc trait ReduceSinkTableDesc extends LogHelper { self: Operator[_ <: HiveDesc] => - + // Seq(tag, (Key TableDesc, Value TableDesc)) def keyValueDescs(): Seq[(Int, (TableDesc, TableDesc))] = { // get the parent ReduceSinkOperator and sort it by tag - var reduceSinkOps = for(op <- self.parentOperators.toSeq; - if(op.isInstanceOf[ReduceSinkOperator])) - yield op.asInstanceOf[ReduceSinkOperator] - - reduceSinkOps.map(f=>(f.getTag, f.getKeyValueTableDescs)) + val reduceSinkOps = + for (op <- self.parentOperators.toSeq; + if (op.isInstanceOf[ReduceSinkOperator])) yield op.asInstanceOf[ReduceSinkOperator] + + reduceSinkOps.map(f => (f.getTag, f.getKeyValueTableDescs)) } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 33506ed6..ddbbd234 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -162,15 +162,12 @@ class ScriptOperator extends UnaryOperator[ScriptDesc] { override def outputObjectInspector() = scriptOutputDeserializer.getObjectInspector() override def initializeOnSlave() { - scriptOutputDeserializer = conf.getScriptOutputInfo() - .getDeserializerClass().newInstance() - scriptOutputDeserializer.initialize(localHconf, conf - .getScriptOutputInfo().getProperties()) + scriptOutputDeserializer = conf.getScriptOutputInfo().getDeserializerClass().newInstance() + scriptOutputDeserializer.initialize(localHconf, conf.getScriptOutputInfo().getProperties()) scriptInputSerializer = conf.getScriptInputInfo().getDeserializerClass() - .newInstance().asInstanceOf[Serializer] - scriptInputSerializer.initialize( - localHconf, conf.getScriptInputInfo().getProperties()) + .newInstance().asInstanceOf[Serializer] + scriptInputSerializer.initialize(localHconf, conf.getScriptInputInfo().getProperties()) } /** @@ -206,7 +203,7 @@ class ScriptOperator extends UnaryOperator[ScriptDesc] { scriptOpHelper.addJobConfToEnvironment(hconf, envs) envs.put(scriptOpHelper.safeEnvVarName(HiveConf.ConfVars.HIVEALIAS.varname), - String.valueOf(alias)) + String.valueOf(alias)) // Create an environment variable that uniquely identifies this script // operator diff --git a/src/main/scala/shark/execution/SelectOperator.scala b/src/main/scala/shark/execution/SelectOperator.scala index 885c19ef..bdbb16c0 100755 --- a/src/main/scala/shark/execution/SelectOperator.scala +++ b/src/main/scala/shark/execution/SelectOperator.scala @@ -50,6 +50,7 @@ class SelectOperator extends UnaryOperator[SelectDesc] { } } } + override def initializeOnSlave() { initializeEvals(true) } @@ -71,9 +72,10 @@ class SelectOperator extends UnaryOperator[SelectDesc] { } override def outputObjectInspector(): ObjectInspector = { - if (conf.isSelStarNoCompute()) + if (conf.isSelStarNoCompute()) { super.outputObjectInspector() - else + } else { initEvaluatorsAndReturnStruct(evals, conf.getOutputColumnNames(), objectInspector) + } } } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 54a313f8..a8ed7537 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -93,16 +93,16 @@ class TableScanOperator extends TopOperator[TableScanDesc] { val partCols = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) val partNames = new ArrayList[String] val partObjectInspectors = new ArrayList[ObjectInspector] - partCols.trim().split("/").foreach{ key => + partCols.trim().split("/").foreach { key => partNames.add(key) partObjectInspectors.add(PrimitiveObjectInspectorFactory.javaStringObjectInspector) } val partObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( - partNames, partObjectInspectors) + partNames, partObjectInspectors) val oiList = Arrays.asList( - tableDeser.getObjectInspector().asInstanceOf[StructObjectInspector], - partObjectInspector.asInstanceOf[StructObjectInspector]) + tableDeser.getObjectInspector().asInstanceOf[StructObjectInspector], + partObjectInspector.asInstanceOf[StructObjectInspector]) // new oi is union of table + partition object inspectors ObjectInspectorFactory.getUnionStructObjectInspector(oiList) } diff --git a/src/main/scala/shark/execution/UDTFOperator.scala b/src/main/scala/shark/execution/UDTFOperator.scala index e680587b..17a39cd5 100755 --- a/src/main/scala/shark/execution/UDTFOperator.scala +++ b/src/main/scala/shark/execution/UDTFOperator.scala @@ -65,7 +65,7 @@ class UDTFOperator extends UnaryOperator[UDTFDesc] { } override def outputObjectInspector() = outputObjInspector - + override def processPartition(split: Int, iter: Iterator[_]): Iterator[_] = { iter.flatMap { row => explode(row) diff --git a/src/main/scala/shark/execution/UnionOperator.scala b/src/main/scala/shark/execution/UnionOperator.scala index 0e2a2270..7df74d9e 100755 --- a/src/main/scala/shark/execution/UnionOperator.scala +++ b/src/main/scala/shark/execution/UnionOperator.scala @@ -58,9 +58,10 @@ class UnionOperator extends NaryOperator[UnionDesc] { // whether we need to do transformation for each parent var parents = parentOperators.length var outputOI = outputObjectInspector() - needsTransform = Array.tabulate[Boolean](objectInspectors.length) { - // ObjectInspectors created by the ObjectInspectorFactory, which take the same ref if equals - i=>objectInspectors(i) != outputOI + needsTransform = Array.tabulate[Boolean](objectInspectors.length) { i => + // ObjectInspectors created by the ObjectInspectorFactory, + // which take the same ref if equals + objectInspectors(i) != outputOI } initializeOnSlave() @@ -89,17 +90,17 @@ class UnionOperator extends NaryOperator[UnionDesc] { val outputFieldOIs = columnTypeResolvers.map(_.get()) outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector( - columnNames, outputFieldOIs.toList) + columnNames, outputFieldOIs.toList) // whether we need to do transformation for each parent // We reuse needsTransform from Hive because the comparison of object // inspectors are hard once we send object inspectors over the wire. needsTransform.zipWithIndex.filter(_._1).foreach { case(transform, p) => logDebug("Union Operator needs to transform row from parent[%d] from %s to %s".format( - p, objectInspectors(p), outputObjInspector)) + p, objectInspectors(p), outputObjInspector)) } } - + override def outputObjectInspector() = outputObjInspector /** diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 6bb6b74e..e3fc8ea0 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -455,8 +455,8 @@ object SharkSemanticAnalyzer extends LogHelper { * Given a Hive top operator (e.g. TableScanOperator), find all the file sink * operators (aka file output operator). */ - private def findAllHiveFileSinkOperators(op: HiveOperator[_<: HiveDesc]): - Seq[HiveOperator[_<: HiveDesc]] = { + private def findAllHiveFileSinkOperators(op: HiveOperator[_<: HiveDesc]) + : Seq[HiveOperator[_<: HiveDesc]] = { if (op.getChildOperators() == null || op.getChildOperators().size() == 0) { Seq[HiveOperator[_<: HiveDesc]](op) } else { From 8d62fbfb94cf01d8da5f61c8f693cbcdb49fac57 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 27 Oct 2013 18:22:44 -0700 Subject: [PATCH 177/331] Account for removed `Operator.hiveOp` in FileSinkOperator. --- src/main/scala/shark/execution/FileSinkOperator.scala | 2 +- src/main/scala/shark/execution/Operator.scala | 2 +- src/main/scala/shark/execution/package.scala | 1 - src/main/scala/shark/parse/SharkSemanticAnalyzer.scala | 2 +- 4 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index b990e1bb..c0f6d868 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -191,7 +191,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { logDebug("Total number of rows written: " + rows.sum) } - hiveOp.jobClose(localHconf, true /* success */, new JobCloseFeedBack) + localHiveOp.jobClose(localHconf, true /* success */, new JobCloseFeedBack) rdd } } diff --git a/src/main/scala/shark/execution/Operator.scala b/src/main/scala/shark/execution/Operator.scala index 7adaed34..31200dce 100755 --- a/src/main/scala/shark/execution/Operator.scala +++ b/src/main/scala/shark/execution/Operator.scala @@ -108,7 +108,7 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { } } - def desc() = _desc + def desc = _desc def setDesc[B >: T](d: B) {_desc = d.asInstanceOf[T]} diff --git a/src/main/scala/shark/execution/package.scala b/src/main/scala/shark/execution/package.scala index eef52208..79bd903f 100755 --- a/src/main/scala/shark/execution/package.scala +++ b/src/main/scala/shark/execution/package.scala @@ -30,4 +30,3 @@ package object execution { implicit def kryoWrapper2object[T](wrapper: KryoSerializationWrapper[T]): T = wrapper.value } - diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index e3fc8ea0..20b837d0 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.storage.StorageLevel import shark.{CachedTableRecovery, LogHelper, SharkConfVars, SharkEnv, Utils} -import shark.execution.{HiveOperator, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator, +import shark.execution.{HiveDesc, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator, SharkDDLWork, SparkWork, TerminalOperator} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} From 13ac9a66ea99e467fa1c0dda7b2d191a26bba837 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Sun, 20 Oct 2013 19:38:47 -0700 Subject: [PATCH 178/331] Handle database namespaces for cached tables Cached tables currently ignore database name which makes caching buggy for users using multiple databases. Enhanced the memoryMetadataManager to require database name inaddition to the tablename for all metadata operations. Conflicts: src/main/scala/shark/SharkEnv.scala src/main/scala/shark/execution/MemoryStoreSinkOperator.scala src/main/scala/shark/execution/SharkDDLTask.scala src/main/scala/shark/execution/TableScanOperator.scala src/main/scala/shark/memstore2/MemoryMetadataManager.scala src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala src/main/scala/shark/parse/SharkSemanticAnalyzer.scala src/test/scala/shark/SQLSuite.scala --- src/main/scala/shark/SharkEnv.scala | 14 +-- .../execution/MemoryStoreSinkOperator.scala | 10 +- .../shark/execution/OperatorFactory.scala | 2 + .../scala/shark/execution/SharkDDLTask.scala | 53 +++++++--- .../shark/execution/TableScanOperator.scala | 28 ++--- .../memstore2/MemoryMetadataManager.scala | 79 ++++++++------ .../parse/SharkDDLSemanticAnalyzer.scala | 9 +- .../shark/parse/SharkSemanticAnalyzer.scala | 10 +- src/test/scala/shark/SQLSuite.scala | 100 +++++++++++++----- 9 files changed, 202 insertions(+), 103 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 2df8a400..6219a37f 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -118,15 +118,16 @@ object SharkEnv extends LogHelper { * * @param tableName The table that should be dropped from the Shark metastore and from memory storage. */ - def dropTable(tableName: String): Option[RDD[_]] = { - if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(tableName)) { - if (SharkEnv.tachyonUtil.dropTable(tableName)) { - logInfo("Table " + tableName + " was deleted from Tachyon."); + def dropTable(databaseName: String, tableName: String): Option[RDD[_]] = { + val tableKey = makeTableKey(databaseName, tableName) + if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(tableKey)) { + if (SharkEnv.tachyonUtil.dropTable(tableKey)) { + logInfo("Table " + tableKey + " was deleted from Tachyon."); } else { - logWarning("Failed to remove table " + tableName + " from Tachyon."); + logWarning("Failed to remove table " + tableKey + " from Tachyon."); } } - return memoryMetadataManager.removeTable(tableName) + return memoryMetadataManager.removeTable(databaseName, tableName) } /** Cleans up and shuts down the Shark environments. */ @@ -141,6 +142,7 @@ object SharkEnv extends LogHelper { /** Return the value of an environmental variable as a string. */ def getEnv(varname: String) = if (System.getenv(varname) == null) "" else System.getenv(varname) + } diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index a0c90bd0..124f31f7 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -53,6 +53,9 @@ class MemoryStoreSinkOperator extends TerminalOperator { // the table that is modified. @BeanProperty var tableName: String = _ + // The Hive metastore DB that the `tableName` table belongs to. + @BeanProperty var databaseName: String = _ + // Used only for commands that target Hive partitions. The partition key is a set of unique values // for the the table's partitioning columns and identifies the partition (represented by an RDD) // that will be created or modified by the INSERT command being handled. @@ -143,8 +146,9 @@ class MemoryStoreSinkOperator extends TerminalOperator { outputRDD, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } else { // Put the table in Spark block manager. - op.logInfo("Putting %sRDD for %s in Spark block manager, %s %s %s %s".format( + op.logInfo("Putting %sRDD for %s.%s in Spark block manager, %s %s %s %s".format( if (useUnionRDD) "Union" else "", + databaseName, tableName, if (storageLevel.deserialized) "deserialized" else "serialized", if (storageLevel.useMemory) "in memory" else "", @@ -209,7 +213,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Combine stats for the two RDDs that were combined into UnionRDD. val numPartitions = statsAcc.value.toMap.size val currentStats = statsAcc.value - SharkEnv.memoryMetadataManager.getStats(tableName) match { + SharkEnv.memoryMetadataManager.getStats(databseName, tableName) match { case Some(otherIndexToStats) => { for ((otherIndex, tableStats) <- otherIndexToStats) { currentStats.append((otherIndex + numPartitions, tableStats)) @@ -223,7 +227,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { } // Get the column statistics back to the cache manager. - SharkEnv.memoryMetadataManager.putStats(tableName, columnStats) + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, columnStats) if (tachyonWriter != null) { tachyonWriter.updateMetadata(ByteBuffer.wrap(JavaSerializer.serialize(columnStats))) diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 6992cdea..66b4d592 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -48,6 +48,7 @@ object OperatorFactory extends LogHelper { def createSharkMemoryStoreOutputPlan( hiveTerminalOp: HOperator[_<:HiveDesc], tableName: String, + databaseName: String, storageLevel: StorageLevel, numColumns: Int, hivePartitionKey: String, @@ -59,6 +60,7 @@ object OperatorFactory extends LogHelper { classOf[MemoryStoreSinkOperator], hiveOp).asInstanceOf[MemoryStoreSinkOperator] sinkOp.localHiveOp = hiveOp sinkOp.tableName = tableName + sinkOp.databaseName = databaseName sinkOp.storageLevel = storageLevel sinkOp.numColumns = numColumns sinkOp.cacheMode = cacheMode diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index ccd73770..f6e9dcaf 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -53,11 +53,16 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] with Serializable with LogHelper { override def execute(driverContext: DriverContext): Int = { + val hiveDb = Hive.get(conf) + + // TODO(harvey): Check whether the `hiveDb` is needed. HiveTask should already have a `db` to + // use. + work.ddlDesc match { - case creatTblDesc: CreateTableDesc => createTable(creatTblDesc, work.cacheMode) - case addPartitionDesc: AddPartitionDesc => addPartition(addPartitionDesc) - case dropTableDesc: DropTableDesc => dropTableOrPartition(dropTableDesc) - case alterTableDesc: AlterTableDesc => alterTable(alterTableDesc) + case creatTblDesc: CreateTableDesc => createTable(hiveDb, creatTblDesc, work.cacheMode) + case addPartitionDesc: AddPartitionDesc => addPartition(hiveDb, addPartitionDesc) + case dropTableDesc: DropTableDesc => dropTableOrPartition(hiveDb, dropTableDesc) + case alterTableDesc: AlterTableDesc => alterTable(hiveDb, alterTableDesc) case _ => { throw new UnsupportedOperationException( "Shark does not require a Shark DDL task for: " + work.ddlDesc.getClass.getName) @@ -70,7 +75,10 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } /** Handles a CREATE TABLE or CTAS. */ - def createTable(createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { + def createTable( + hiveMetadataDb: Hive, + createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { + val dbName = hiveMetadataDb.getCurrentDatabase() val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps @@ -81,6 +89,7 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] // Add a new PartitionedMemoryTable entry in the Shark metastore. // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( + dbName, tableName, cacheMode, preferredStorageLevel, @@ -94,12 +103,15 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } /** Handles an ALTER TABLE ADD PARTITION. */ - def addPartition(addPartitionDesc: AddPartitionDesc) { + def addPartition( + hiveMetadataDb: Hive, + addPartitionDesc: AddPartitionDesc) { + val dbName = hiveMetadataDb.getCurrentDatabase() val tableName = addPartitionDesc.getTableName - val partitionedTable = getPartitionedTableWithAssertions(tableName) + val partitionedTable = getPartitionedTableWithAssertions(dbName, tableName) // Find the set of partition column values that specifies the partition being added. - val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val hiveTable = db.getTable(tableName, false /* throwException */); val partCols: Seq[String] = hiveTable.getPartCols.map(_.getName) val partColToValue: JavaMap[String, String] = addPartitionDesc.getPartSpec // String format for partition key: 'col1=value1/col2=value2/...' @@ -111,17 +123,20 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] * A DropTableDesc is used for both dropping entire tables (i.e., DROP TABLE) and for dropping * individual partitions of a table (i.e., ALTER TABLE DROP PARTITION). */ - def dropTableOrPartition(dropTableDesc: DropTableDesc) { + def dropTableOrPartition( + hiveMetadataDb: Hive, + dropTableDesc: DropTableDesc) { + val dbName = hiveMetadataDb.getCurrentDatabase() val tableName = dropTableDesc.getTableName - val hiveTable = db.getTable(db.getCurrentDatabase(), tableName, false /* throwException */); + val hiveTable = db.getTable(tableName, false /* throwException */); val partSpecs: JavaList[PartitionSpec] = dropTableDesc.getPartSpecs if (partSpecs == null) { // The command is a true DROP TABLE. - SharkEnv.dropTable(tableName) + SharkEnv.dropTable(dbName, tableName) } else { // The command is an ALTER TABLE DROP PARTITION - val partitionedTable = getPartitionedTableWithAssertions(tableName) + val partitionedTable = getPartitionedTableWithAssertions(dbName, tableName) // Find the set of partition column values that specifies the partition being dropped. val partCols: Seq[String] = hiveTable.getPartCols.map(_.getName) for (partSpec <- partSpecs) { @@ -134,12 +149,15 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } /** Handles miscellaneous ALTER TABLE 'tableName' commands. */ - def alterTable(alterTableDesc: AlterTableDesc) { + def alterTable( + hiveMetadataDb: Hive, + alterTableDesc: AlterTableDesc) { + val dbName = hiveMetadataDb.getCurrentDatabase() alterTableDesc.getOp() match { case AlterTableDesc.AlterTableTypes.RENAME => { val oldName = alterTableDesc.getOldName val newName = alterTableDesc.getNewName - SharkEnv.memoryMetadataManager.renameTable(oldName, newName) + SharkEnv.memoryMetadataManager.renameTable(dbName, oldName, newName) } case _ => { // TODO(harvey): Support more ALTER TABLE commands, such as ALTER TABLE PARTITION RENAME TO. @@ -149,10 +167,12 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } } - def getPartitionedTableWithAssertions(tableName: String): PartitionedMemoryTable = { + private def getPartitionedTableWithAssertions( + dbName: String, + tableName: String): PartitionedMemoryTable = { // Sanity checks: make sure that the table we're modifying exists in the Shark metastore and // is actually partitioned. - val tableOpt = SharkEnv.memoryMetadataManager.getTable(tableName) + val tableOpt = SharkEnv.memoryMetadataManager.getTable(dbName, tableName) assert(tableOpt.isDefined, "Internal Error: table %s doesn't exist in Shark metastore.") assert(tableOpt.get.isInstanceOf[PartitionedMemoryTable], "Internal Error: table %s isn't partitioned when it should be.") @@ -164,4 +184,5 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] override def getName = "DDL-SPARK" override def localizeMRTmpFilesImpl(ctx: Context) = Unit + } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index a8ed7537..e995232b 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -110,8 +110,10 @@ class TableScanOperator extends TopOperator[TableScanDesc] { override def execute(): RDD[_] = { assert(parentOperators.size == 0) - val tableKey: String = tableDesc.getTableName.split('.')(1) + val tableNameSplit = tableName.split('.') // Split from 'databaseName.tableName' + val tableName = tableNameSplit(1) + val databaseName = tableNameSplit(0) // There are three places we can load the table from. // 1. Tachyon table // 2. Spark heap (block manager), accessed through the Shark MemoryMetadataManager @@ -121,19 +123,19 @@ class TableScanOperator extends TopOperator[TableScanDesc] { // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). - if (!SharkEnv.memoryMetadataManager.containsTable(tableKey)) { + if (!SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { logError("""|Table %s not found in block manager. |Are you trying to access a cached table from a Shark session other than |the one in which it was created?""".stripMargin.format(tableKey)) - throw(new QueryExecutionException("Cached table not found")) + throw new QueryExecutionException("Cached table not found") } - if (SharkEnv.memoryMetadataManager.isHivePartitioned(tableKey)) { + if (SharkEnv.memoryMetadataManager.isHivePartitioned(databaseName, tableName)) { // Get the union of RDDs repesenting the selected Hive partition(s). - makeCachedPartitionRDD(tableKey, parts) + return makeCachedPartitionRDD(tableKey, parts) } else { - val rdd = SharkEnv.memoryMetadataManager.getMemoryTable(tableKey).get.tableRDD - logInfo("Loading table " + tableKey + " from Spark block manager") - createPrunedRdd(tableKey, rdd) + val table = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName).get + logInfo("Loading table %s.%s from Spark block manager".format(databaseName, tableName)) + return createPrunedRdd(databaseNamem, tableName, table.tableRDD) } } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. @@ -143,26 +145,26 @@ class TableScanOperator extends TopOperator[TableScanDesc] { logInfo("Loading table " + tableKey + " from Tachyon.") var indexToStats: collection.Map[Int, TablePartitionStats] = - SharkEnv.memoryMetadataManager.getStats(tableKey).getOrElse(null) + SharkEnv.memoryMetadataManager.getStats(databaseName, tableKey).getOrElse(null) if (indexToStats == null) { val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( statsByteBuffer.array()) logInfo("Loading table " + tableKey + " stats from Tachyon.") - SharkEnv.memoryMetadataManager.putStats(tableKey, indexToStats) + SharkEnv.memoryMetadataManager.putStats(databaseName, tableKey, indexToStats) } - return createPrunedRdd(tableKey, SharkEnv.tachyonUtil.createRDD(tableKey)) + return createPrunedRdd(databaseName, tableName, SharkEnv.tachyonUtil.createRDD(tableName)) } else { // Table is a Hive table on HDFS (or other Hadoop storage). return makeRDDFromHadoop() } } - private def createPrunedRdd(tableKey: String, rdd: RDD[_]): RDD[_] = { + private def createPrunedRdd(databaseName: String, tableName: String, rdd: RDD[_]): RDD[_] = { // Stats used for map pruning. val indexToStats: collection.Map[Int, TablePartitionStats] = - SharkEnv.memoryMetadataManager.getStats(tableKey).get + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName).get // Run map pruning if the flag is set, there exists a filter predicate on // the input table and we have statistics on the table. diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 8c0e0382..231573f3 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -40,32 +40,54 @@ class MemoryMetadataManager { private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - def putStats(key: String, stats: collection.Map[Int, TablePartitionStats]) { - _keyToStats.put(key.toLowerCase, stats) + def putStats( + databaseName: String, + tableName: String, + stats: collection.Map[Int, TablePartitionStats]) { + val tableKey = makeTableKey(databaseName, tableKey) + _keyToStats.put(tableKey, stats) + } + + def getStats( + databaseName: String, + tableName: String): Option[collection.Map[Int, TablePartitionStats]] = { + val tableKey = makeTableKey(databaseName, tableName) + _keyToStats.get(tableKey) + } + + def isHivePartitioned(databaseName: String, tableName: String): Boolean = { + val tableKey = makeTableKey(databaseName, tableName) + _keyToTable.get(tableKey) match { + case Some(table) => return table.isInstanceOf[PartitionedMemoryTable] + case None => return false + } } - def getStats(key: String): Option[collection.Map[Int, TablePartitionStats]] = { - _keyToStats.get(key.toLowerCase) + def containsTable(databaseName: String, tableName: String): Boolean = { + _keyToTable.contains(makeTableKey(databaseName, tableName)) } def createMemoryTable( + databaseName: String, tableName: String, cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel ): MemoryTable = { - var newTable = new MemoryTable(tableName.toLowerCase, cacheMode, preferredStorageLevel) + val tableKey = makeTableKey(databaseName, tableName) + var newTable = new MemoryTable(tableKey, cacheMode, preferredStorageLevel) _keyToTable.put(tableName.toLowerCase, newTable) return newTable } def createPartitionedMemoryTable( + databaseName: String, tableName: String, cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, tblProps: JavaMap[String, String] ): PartitionedMemoryTable = { - var newTable = new PartitionedMemoryTable( - tableName.toLowerCase, cacheMode, preferredStorageLevel) + val tableKey = makeTableKey(databaseName, tableName) + var newTable = new PartitionedMemoryTable(tableKey, cacheMode, preferredStorageLevel) // Determine the cache policy to use and read any user-specified cache settings. val cachePolicyStr = tblProps.getOrElse(SharkConfVars.CACHE_POLICY.varname, SharkConfVars.CACHE_POLICY.defaultVal) @@ -73,19 +95,10 @@ class MemoryMetadataManager { SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toInt newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize) - _keyToTable.put(tableName.toLowerCase, newTable) + _keyToTable.put(tableKey, newTable) return newTable } - def isHivePartitioned(tableName: String): Boolean = { - _keyToTable.get(tableName.toLowerCase) match { - case Some(table) => return table.isInstanceOf[PartitionedMemoryTable] - case None => return false - } - } - - def containsTable(tableName: String): Boolean = _keyToTable.contains(tableName.toLowerCase) - def getTable(tableName: String): Option[Table] = _keyToTable.get(tableName.toLowerCase) def getMemoryTable(tableName: String): Option[MemoryTable] = { @@ -106,17 +119,17 @@ class MemoryMetadataManager { tableOpt.asInstanceOf[Option[PartitionedMemoryTable]] } - def renameTable(oldName: String, newName: String) { - val lowerCaseOldName = oldName.toLowerCase - if (containsTable(lowerCaseOldName)) { - val lowerCaseNewName = newName.toLowerCase + def renameTable(databaseName: String, oldName: String, newName: String) { + if (containsTable(databaseName, oldName)) { + val oldTableKey = makeTableKey(databaseName, oldName) + val newTableKey = makeTableKey(databaseName, newName) - val statsValueEntry = _keyToStats.remove(lowerCaseOldName).get - val tableValueEntry = _keyToTable.remove(lowerCaseOldName).get - tableValueEntry.tableName = lowerCaseNewName + val statsValueEntry = _keyToStats.remove(oldTableKey).get + val tableValueEntry = _keyToTable.remove(oldTableKey).get + tableValueEntry.tableName = newTableKey - _keyToStats.put(lowerCaseNewName, statsValueEntry) - _keyToTable.put(lowerCaseNewName, tableValueEntry) + _keyToStats.put(newTableKey, statsValueEntry) + _keyToTable.put(newTableKey, tableValueEntry) } } @@ -129,13 +142,12 @@ class MemoryMetadataManager { * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ - def removeTable(tableName: String): Option[RDD[_]] = { - val lowerCaseTableName = tableName.toLowerCase + def removeTable(databaseName:String, tableName: String): Option[RDD[_]] = { + val tableKey = makeTableKey(databaseName, tableName) // Remove MemoryTable's entry from Shark metadata. - _keyToStats.remove(lowerCaseTableName) - - val tableValue: Option[Table] = _keyToTable.remove(lowerCaseTableName) + _keyToStats.remove(tableKey) + val tableValue: Option[Table] = _keyToTable.remove(tableKey) return tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) } @@ -147,6 +159,11 @@ class MemoryMetadataManager { def getAllKeyStrings(): Seq[String] = { _keyToTable.keys.collect { case k: String => k } toSeq } + + private def makeTableKey(databaseName: String, tableName: String) = { + return (databaseName + '.' + tableName).toLowerCase + } + } diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 5a113fb5..97a55c8c 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -37,9 +37,10 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) } def analyzeDropTableOrDropParts(ast: ASTNode) { + val databaseName = db.getCurrentDatabase() val tableName = getTableName(ast) // Create a SharkDDLTask only if the table is cached. - if (SharkEnv.memoryMetadataManager.containsTable(tableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks // and DDLWorks that contain DropTableDesc objects. for (ddlTask <- rootTasks) { @@ -51,9 +52,10 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) } def analyzeAlterTableAddParts(ast: ASTNode) { + val databaseName = db.getCurrentDatabase() val tableName = getTableName(ast) // Create a SharkDDLTask only if the table is cached. - if (SharkEnv.memoryMetadataManager.containsTable(tableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks // and DDLWorks that contain AddPartitionDesc objects. for (ddlTask <- rootTasks) { @@ -65,8 +67,9 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) } private def analyzeAlterTableRename(astNode: ASTNode) { + val databaseName = db.getCurrentDatabase() val oldTableName = getTableName(astNode) - if (SharkEnv.memoryMetadataManager.containsTable(oldTableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(databaseName, oldTableName)) { val newTableName = BaseSemanticAnalyzer.getUnescapedName( astNode.getChild(1).asInstanceOf[ASTNode]) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 20b837d0..5cda8635 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -164,12 +164,16 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) } else { // Otherwise, check if we are inserting into a table that was cached. + val tableNameSplit = tableName.split('.') // Split from 'databaseName.tableName' + val cachedTableName = tableNameSplit(1) + val databaseName = tableNameSplit(0) val cachedTableName = tableName.split('.')(1) // Ignore the database name - if (SharkEnv.memoryMetadataManager.containsTable(cachedTableName)) { + if (SharkEnv.memoryMetadataManager.containsTable(databaseName, cachedTableName)) { if (hiveSinkOps.size == 1) { // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) - val table = SharkEnv.memoryMetadataManager.getTable(cachedTableName).get + val table = SharkEnv.memoryMetadataManager.getTable( + databaseName, cachedTableName).get val cacheMode = table.cacheMode var hivePartitionKey = new String if (SharkEnv.memoryMetadataManager.isHivePartitioned(cachedTableName)) { @@ -183,6 +187,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, cachedTableName, + databaseName, preferredStorageLevel, _resSchema.size, /* numColumns */ hivePartitionKey, @@ -209,6 +214,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOps.head, qb.getTableDesc.getTableName, + qb.getTableDesc.getDatabaseName, preferredStorageLevel, _resSchema.size, /* numColumns */ new String, /* hivePartitionKey */ diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 1d8ffcc0..2deae36b 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -30,6 +30,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val WAREHOUSE_PATH = TestUtils.getWarehousePath() val METASTORE_PATH = TestUtils.getMetastorePath() val MASTER = "local" + val DEFAULT_DB_NAME = "default" var sc: SharkContext = _ @@ -42,6 +43,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) + // second db + sc.sql("create database if not exists seconddb") + // test sc.runSql("drop table if exists test") sc.runSql("CREATE TABLE test (key INT, val STRING)") @@ -127,8 +131,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { select * from test_cached""".format(tableName, partitionNum)) partitionNum += 1 } - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get return partitionedTable } @@ -256,16 +261,18 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("Use regular CREATE TABLE and '_cached' suffix to create cached table") { sc.runSql("drop table if exists empty_table_cached") sc.runSql("create table empty_table_cached(key string, value string)") - assert(SharkEnv.memoryMetadataManager.containsTable("empty_table_cached")) - assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "empty_table_cached")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned(DEFAULT_DB_NAME, "empty_table_cached")) } test("Use regular CREATE TABLE and table properties to create cached table") { sc.runSql("drop table if exists empty_table_cached_tbl_props") sc.runSql("""create table empty_table_cached_tbl_props(key string, value string) TBLPROPERTIES('shark.cache' = 'true')""") - assert(SharkEnv.memoryMetadataManager.containsTable("empty_table_cached_tbl_props")) - assert(!SharkEnv.memoryMetadataManager.isHivePartitioned("empty_table_cached_tbl_props")) + assert(SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "empty_table_cached_tbl_props")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned( + DEFAULT_DB_NAME, "empty_table_cached_tbl_props")) } test("Insert into empty cached table") { @@ -280,8 +287,10 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists test_rename") sc.runSql("create table test_oldname_cached as select * from test") sc.runSql("alter table test_oldname_cached rename to test_rename") - assert(!SharkEnv.memoryMetadataManager.containsTable("test_oldname_cached")) - assert(SharkEnv.memoryMetadataManager.containsTable("test_rename")) + + assert(!SharkEnv.memoryMetadataManager.contains(DEFAULT_DB_NAME, "test_oldname_cached")) + assert(SharkEnv.memoryMetadataManager.contains(DEFAULT_DB_NAME, "test_rename")) + expectSql("select count(*) from test_rename", "500") } @@ -317,7 +326,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists ctas_tbl_props") sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='true') as select * from test""") - assert(SharkEnv.memoryMetadataManager.containsTable("ctas_tbl_props")) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "ctas_tbl_props")) expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") } @@ -327,7 +336,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { CREATE TABLE ctas_tbl_props_result_should_not_be_cached TBLPROPERTIES ('shark.cache'='false') AS select * from test""") - assert(!SharkEnv.memoryMetadataManager.containsTable("ctas_tbl_props_should_not_be_cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "ctas_tbl_props_should_not_be_cached")) } test("cached tables with complex types") { @@ -351,7 +361,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(sc.sql("select d from test_complex_types_cached where a = 'a0'").head === """{"d01":["d011","d012"],"d02":["d021","d022"]}""") - assert(SharkEnv.memoryMetadataManager.containsTable("test_complex_types_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "test_complex_types_cached")) } test("disable caching by default") { @@ -359,7 +370,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists should_not_be_cached") sc.runSql("create table should_not_be_cached as select * from test") expectSql("select key from should_not_be_cached where key = 407", "407") - assert(!SharkEnv.memoryMetadataManager.containsTable("should_not_be_cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "should_not_be_cached")) sc.runSql("set shark.cache.flag.checkTableName=true") } @@ -368,7 +380,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as select * from test""") expectSql("select val from sharktest5Cached where key = 407", "val_407") - assert(SharkEnv.memoryMetadataManager.containsTable("sharkTest5Cached")) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "sharkTest5Cached")) } test("dropping cached tables should clean up RDDs") { @@ -376,7 +388,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as select * from test""") sc.runSql("drop table sharkTest5Cached") - assert(!SharkEnv.memoryMetadataManager.containsTable("sharkTest5Cached")) + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "sharkTest5Cached")) } ////////////////////////////////////////////////////////////////////////////// @@ -387,16 +399,19 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists empty_part_table_cached") sc.runSql("""create table empty_part_table_cached(key int, value string) partitioned by (keypart int)""") - assert(SharkEnv.memoryMetadataManager.containsTable("empty_part_table_cached")) - assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "empty_part_table_cached")) + assert(SharkEnv.memoryMetadataManager.isHivePartitioned( + DEFAULT_DB_NAME, "empty_part_table_cached")) } test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { sc.runSql("drop table if exists empty_part_table_cached_tbl_props") sc.runSql("""create table empty_part_table_cached_tbl_props(key int, value string) partitioned by (keypart int) tblproperties('shark.cache' = 'true')""") - assert(SharkEnv.memoryMetadataManager.containsTable("empty_part_table_cached_tbl_props")) - assert(SharkEnv.memoryMetadataManager.isHivePartitioned("empty_part_table_cached_tbl_props")) + assert(SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "empty_part_table_cached_tbl_props")) + assert(SharkEnv.memoryMetadataManager.isHivePartitioned( + DEFAULT_DB_NAME, "empty_part_table_cached_tbl_props")) } test("alter cached table by adding a new partition") { @@ -406,8 +421,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""alter table alter_part_cached add partition(keypart = 1)""") val tableName = "alter_part_cached" val partitionColumn = "keypart=1" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get assert(partitionedTable.containsPartition(partitionColumn)) } @@ -418,8 +434,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("""alter table alter_drop_part_cached add partition(keypart = 1)""") val tableName = "alter_drop_part_cached" val partitionColumn = "keypart=1" - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get assert(partitionedTable.containsPartition(partitionColumn)) sc.runSql("""alter table alter_drop_part_cached drop partition(keypart = 1)""") assert(!partitionedTable.containsPartition(partitionColumn)) @@ -496,7 +513,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") val keypart1RDD = partitionedTable.getPartition("keypart=1") sc.runSql("drop table drop_union_part_cached") - assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) // All RDDs should have been unpersisted. assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) } @@ -509,8 +526,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val tableName = "default_policy_cached" sc.runSql("""create table default_policy_cached(key int, value string) partitioned by (keypart int)""") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get val cachePolicy = partitionedTable.cachePolicy assert(cachePolicy.isInstanceOf[shark.memstore2.CacheAllPolicy[_, _]]) } @@ -566,7 +584,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* numPartitionsToCreate */, 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) @@ -590,7 +608,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* numPartitionsToCreate */, 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) @@ -614,7 +632,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* numPartitionsToCreate */, 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) sc.runSql("alter table evict_partitions_removals drop partition(keypart = 1)") sc.runSql("""insert into table evict_partitions_removals partition(keypart = 4) select * from test""") @@ -631,7 +649,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { 3 /* numPartitionsToCreate */, 3 /* maxCacheSize */, "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(tableName)) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") val lvl = TestUtils.getStorageLevelOfRDD(keypart1RDD.get) assert(lvl == StorageLevel.MEMORY_AND_DISK, "got: " + lvl) @@ -684,6 +702,27 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { where year(from_unixtime(k)) between "2013" and "2014" """, Array[String]("0")) } + ////////////////////////////////////////////////////////////////////////////// + // SharkContext APIs (e.g. sql2rdd, sql) + ////////////////////////////////////////////////////////////////////////////// + + test("cached table in different new database") { + sc.sql("drop table if exists selstar") + sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as + select * from default.test """) + sc.sql("use seconddb") + sc.sql("drop table if exists selstar") + sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as + select * from default.test where key != 'val_487' """) + + sc.sql("use default") + expectSql("select * from selstar where val='val_487'","487 val_487") + + assert(SharkEnv.memoryMetadataManager.contains(DEFAULT_DB_NAME, "selstar")) + assert(SharkEnv.memoryMetadataManager.contains("seconddb", "selstar")) + + } + ////////////////////////////////////////////////////////////////////////////// // various data types ////////////////////////////////////////////////////////////////////////////// @@ -774,4 +813,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val e = intercept[QueryExecutionException] { sc.sql2rdd("asdfasdfasdfasdf") } e.getMessage.contains("semantic") } + + + } From 0b9ccbc788f3004acbd7ee01c654bdd4dc33b175 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 28 Oct 2013 13:32:29 -0700 Subject: [PATCH 179/331] Patches for the 'database namespaces for cached tables' merge. --- src/main/scala/shark/SharkEnv.scala | 2 +- .../execution/MemoryStoreSinkOperator.scala | 22 ++++++++------- .../scala/shark/execution/SharkDDLTask.scala | 2 +- .../shark/execution/TableScanOperator.scala | 27 +++++++++++-------- .../memstore2/MemoryMetadataManager.scala | 24 ++++++++++------- .../shark/parse/SharkSemanticAnalyzer.scala | 5 ++-- src/test/scala/shark/SQLSuite.scala | 13 ++++----- 7 files changed, 54 insertions(+), 41 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 6219a37f..413c0349 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -119,7 +119,7 @@ object SharkEnv extends LogHelper { * @param tableName The table that should be dropped from the Shark metastore and from memory storage. */ def dropTable(databaseName: String, tableName: String): Option[RDD[_]] = { - val tableKey = makeTableKey(databaseName, tableName) + val tableKey = databaseName + "." + tableName if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(tableKey)) { if (SharkEnv.tachyonUtil.dropTable(tableKey)) { logInfo("Table " + tableKey + " was deleted from Tachyon."); diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 124f31f7..7eec7f9c 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -123,7 +123,8 @@ class MemoryStoreSinkOperator extends TerminalOperator { } } - val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned(tableName) + val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( + databaseName, tableName) // If true, a UnionRDD will be used to combine the RDD that contains the query output with the // previous RDD, which is fetched using 'tableName' or - if the table is Hive-partitioned - a @@ -132,7 +133,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { if (tachyonWriter != null) { // Put the table in Tachyon. - op.logInfo("Putting RDD for %s in Tachyon".format(tableName)) + op.logInfo("Putting RDD for %s.%s in Tachyon".format(databaseName, tableName)) tachyonWriter.createTable(ByteBuffer.allocate(0)) outputRDD = outputRDD.mapPartitionsWithIndex { case(part, iter) => val partition = iter.next() @@ -162,10 +163,11 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Handle an INSERT INTO command. var previousRDDOpt: Option[RDD[TablePartition]] = if (isHivePartitioned) { - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + databaseName, tableName).get partitionedTable.getPartition(hivePartitionKey) } else { - SharkEnv.memoryMetadataManager.getMemoryTable(tableName).map(_.tableRDD) + SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName).map(_.tableRDD) } outputRDD = previousRDDOpt match { case Some(previousRDD) => { @@ -186,8 +188,9 @@ class MemoryStoreSinkOperator extends TerminalOperator { } if (isHivePartitioned) { - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableName).get - outputRDD.setName(tableName + "(" + hivePartitionKey + ")") + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + databaseName, tableName).get + outputRDD.setName("%s.%s(%s)".format(databaseName, tableName, hivePartitionKey)) if (useUnionRDD && hasPreviousRDDForUnion) { // An RDD for the Hive partition already exists, so update its metadata entry in // 'partitionedTable'. @@ -200,8 +203,9 @@ class MemoryStoreSinkOperator extends TerminalOperator { } else { outputRDD.setName(tableName) // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). - val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(tableName).getOrElse( - SharkEnv.memoryMetadataManager.createMemoryTable(tableName, cacheMode, storageLevel)) + var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) + .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( + databaseName, tableName, cacheMode, storageLevel)) memoryTable.tableRDD = outputRDD } @@ -213,7 +217,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Combine stats for the two RDDs that were combined into UnionRDD. val numPartitions = statsAcc.value.toMap.size val currentStats = statsAcc.value - SharkEnv.memoryMetadataManager.getStats(databseName, tableName) match { + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { case Some(otherIndexToStats) => { for ((otherIndex, tableStats) <- otherIndexToStats) { currentStats.append((otherIndex + numPartitions, tableStats)) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index f6e9dcaf..161a586a 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -96,7 +96,7 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] tblProps) } else { val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - tableName, cacheMode, preferredStorageLevel) + dbName, tableName, cacheMode, preferredStorageLevel) // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. newTable.tableRDD = new EmptyRDD(SharkEnv.sc) } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index e995232b..c6fb1645 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -111,9 +111,9 @@ class TableScanOperator extends TopOperator[TableScanDesc] { override def execute(): RDD[_] = { assert(parentOperators.size == 0) - val tableNameSplit = tableName.split('.') // Split from 'databaseName.tableName' - val tableName = tableNameSplit(1) + val tableNameSplit = tableDesc.getTableName.split('.') // Split from 'databaseName.tableName' val databaseName = tableNameSplit(0) + val tableName = tableNameSplit(1) // There are three places we can load the table from. // 1. Tachyon table // 2. Spark heap (block manager), accessed through the Shark MemoryMetadataManager @@ -124,35 +124,36 @@ class TableScanOperator extends TopOperator[TableScanDesc] { if (cacheMode == CacheType.HEAP) { // Table should be in Spark heap (block manager). if (!SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { - logError("""|Table %s not found in block manager. - |Are you trying to access a cached table from a Shark session other than - |the one in which it was created?""".stripMargin.format(tableKey)) + logError("""|Table %s.%s not found in block manager. + |Are you trying to access a cached table from a Shark session other than the one + |in which it was created?""".stripMargin.format(databaseName, tableName)) throw new QueryExecutionException("Cached table not found") } if (SharkEnv.memoryMetadataManager.isHivePartitioned(databaseName, tableName)) { // Get the union of RDDs repesenting the selected Hive partition(s). - return makeCachedPartitionRDD(tableKey, parts) + return makeCachedPartitionRDD(databaseName, tableName, parts) } else { val table = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName).get logInfo("Loading table %s.%s from Spark block manager".format(databaseName, tableName)) - return createPrunedRdd(databaseNamem, tableName, table.tableRDD) + return createPrunedRdd(databaseName, tableName, table.tableRDD) } } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. + val tableKey = databaseName + "." + tableName if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") } logInfo("Loading table " + tableKey + " from Tachyon.") var indexToStats: collection.Map[Int, TablePartitionStats] = - SharkEnv.memoryMetadataManager.getStats(databaseName, tableKey).getOrElse(null) + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName).getOrElse(null) if (indexToStats == null) { val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( statsByteBuffer.array()) logInfo("Loading table " + tableKey + " stats from Tachyon.") - SharkEnv.memoryMetadataManager.putStats(databaseName, tableKey, indexToStats) + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, indexToStats) } return createPrunedRdd(databaseName, tableName, SharkEnv.tachyonUtil.createRDD(tableName)) } else { @@ -279,7 +280,10 @@ class TableScanOperator extends TopOperator[TableScanDesc] { * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. */ - private def makeCachedPartitionRDD(tableKey: String, partitions: Array[Partition]): RDD[_] = { + private def makeCachedPartitionRDD( + databaseName: String, + tableName: String, + partitions: Array[Partition]): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) // Get partition field info @@ -297,7 +301,8 @@ class TableScanOperator extends TopOperator[TableScanDesc] { } val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) - val hivePartitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable(tableKey).get + val hivePartitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + databaseName, tableName).get val hivePartitionRDD = hivePartitionedTable.getPartition(partitionKeyStr) hivePartitionRDD.get.mapPartitions { iter => diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 231573f3..ca64fd27 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -44,7 +44,7 @@ class MemoryMetadataManager { databaseName: String, tableName: String, stats: collection.Map[Int, TablePartitionStats]) { - val tableKey = makeTableKey(databaseName, tableKey) + val tableKey = makeTableKey(databaseName, tableName) _keyToStats.put(tableKey, stats) } @@ -75,7 +75,7 @@ class MemoryMetadataManager { ): MemoryTable = { val tableKey = makeTableKey(databaseName, tableName) var newTable = new MemoryTable(tableKey, cacheMode, preferredStorageLevel) - _keyToTable.put(tableName.toLowerCase, newTable) + _keyToTable.put(tableKey, newTable) return newTable } @@ -99,10 +99,13 @@ class MemoryMetadataManager { return newTable } - def getTable(tableName: String): Option[Table] = _keyToTable.get(tableName.toLowerCase) + def getTable(databaseName: String, tableName: String): Option[Table] = { + _keyToTable.get(makeTableKey(databaseName, tableName)) + } - def getMemoryTable(tableName: String): Option[MemoryTable] = { - val tableOpt = _keyToTable.get(tableName.toLowerCase) + def getMemoryTable(databaseName: String, tableName: String): Option[MemoryTable] = { + val tableKey = makeTableKey(databaseName, tableName) + val tableOpt = _keyToTable.get(tableKey) if (tableOpt.isDefined) { assert(tableOpt.get.isInstanceOf[MemoryTable], "getMemoryTable() called for a partitioned table.") @@ -110,8 +113,11 @@ class MemoryMetadataManager { tableOpt.asInstanceOf[Option[MemoryTable]] } - def getPartitionedTable(tableName: String): Option[PartitionedMemoryTable] = { - val tableOpt = _keyToTable.get(tableName.toLowerCase) + def getPartitionedTable( + databaseName: String, + tableName: String): Option[PartitionedMemoryTable] = { + val tableKey = makeTableKey(databaseName, tableName) + val tableOpt = _keyToTable.get(tableKey) if (tableOpt.isDefined) { assert(tableOpt.get.isInstanceOf[PartitionedMemoryTable], "getPartitionedTable() called for a non-partitioned table.") @@ -160,8 +166,8 @@ class MemoryMetadataManager { _keyToTable.keys.collect { case k: String => k } toSeq } - private def makeTableKey(databaseName: String, tableName: String) = { - return (databaseName + '.' + tableName).toLowerCase + private def makeTableKey(databaseName: String, tableName: String): String = { + (databaseName + '.' + tableName).toLowerCase } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 5cda8635..9aa2bdcb 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -167,7 +167,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val tableNameSplit = tableName.split('.') // Split from 'databaseName.tableName' val cachedTableName = tableNameSplit(1) val databaseName = tableNameSplit(0) - val cachedTableName = tableName.split('.')(1) // Ignore the database name if (SharkEnv.memoryMetadataManager.containsTable(databaseName, cachedTableName)) { if (hiveSinkOps.size == 1) { // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. @@ -176,7 +175,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with databaseName, cachedTableName).get val cacheMode = table.cacheMode var hivePartitionKey = new String - if (SharkEnv.memoryMetadataManager.isHivePartitioned(cachedTableName)) { + val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( + databaseName, cachedTableName) + if (isHivePartitioned) { if (cacheMode == CacheType.TACHYON) { throw new SemanticException( "Shark does not support caching Hive-partitioned table(s) in Tachyon.") diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 2deae36b..3f56d5e8 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -288,8 +288,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("create table test_oldname_cached as select * from test") sc.runSql("alter table test_oldname_cached rename to test_rename") - assert(!SharkEnv.memoryMetadataManager.contains(DEFAULT_DB_NAME, "test_oldname_cached")) - assert(SharkEnv.memoryMetadataManager.contains(DEFAULT_DB_NAME, "test_rename")) + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "test_oldname_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "test_rename")) expectSql("select count(*) from test_rename", "500") } @@ -496,7 +496,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val keypart2RDD = partitionedTable.getPartition("keypart=2") val keypart3RDD = partitionedTable.getPartition("keypart=3") sc.runSql("drop table drop_mult_part_cached ") - assert(!SharkEnv.memoryMetadataManager.containsTable(tableName)) + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) // All RDDs should have been unpersisted. assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) @@ -718,8 +718,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("use default") expectSql("select * from selstar where val='val_487'","487 val_487") - assert(SharkEnv.memoryMetadataManager.contains(DEFAULT_DB_NAME, "selstar")) - assert(SharkEnv.memoryMetadataManager.contains("seconddb", "selstar")) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "selstar")) + assert(SharkEnv.memoryMetadataManager.containsTable("seconddb", "selstar")) } @@ -813,7 +813,4 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val e = intercept[QueryExecutionException] { sc.sql2rdd("asdfasdfasdfasdf") } e.getMessage.contains("semantic") } - - - } From 6731d97e4adfa662df3b8220393aa84c07b58426 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 28 Oct 2013 13:32:55 -0700 Subject: [PATCH 180/331] Account for database namespaces in tables created by RDDTable/RDDTableFunctions. --- src/main/scala/shark/api/RDDTableFunctions.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 837bad1c..43c686a6 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -19,12 +19,14 @@ package shark.api import scala.collection.mutable.ArrayBuffer -import shark.SharkEnv -import shark.memstore2.{CacheType, TablePartitionStats, TablePartition, TablePartitionBuilder} -import shark.util.HiveUtils +import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.spark.rdd.RDD +import shark.{SharkContext, SharkEnv} +import shark.memstore2.{CacheType, TablePartitionStats, TablePartition, TablePartitionBuilder} +import shark.util.HiveUtils + class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { @@ -57,10 +59,11 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. + val databaseName = Hive.get(SharkContext.hiveconf).getCurrentDatabase() if (isSucessfulCreateTable) { // Create an entry in the MemoryMetadataManager. val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - tableName, CacheType.HEAP, rdd.getStorageLevel) + databaseName, tableName, CacheType.HEAP, rdd.getStorageLevel) newTable.tableRDD = rdd try { // Force evaluate to put the data in memory. @@ -71,13 +74,13 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { // exception message should already be printed to the console by DDLTask#execute(). HiveUtils.dropTableInHive(tableName) // Drop the table entry from MemoryMetadataManager. - SharkEnv.memoryMetadataManager.removeTable(tableName) + SharkEnv.memoryMetadataManager.removeTable(databaseName, tableName) isSucessfulCreateTable = false } } // Gather the partition statistics. - SharkEnv.memoryMetadataManager.putStats(tableName, statsAcc.value.toMap) + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, statsAcc.value.toMap) } return isSucessfulCreateTable } From 3c50f2471a40065e72e85135d7e36a7a0ae8d908 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 28 Oct 2013 13:37:02 -0700 Subject: [PATCH 181/331] Add a method in SharkEnv to help with formatting table identifiers for RDDs stored in Tachyon. --- src/main/scala/shark/SharkEnv.scala | 10 +++++++++- src/main/scala/shark/execution/TableScanOperator.scala | 2 +- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 413c0349..c8b28706 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -119,7 +119,7 @@ object SharkEnv extends LogHelper { * @param tableName The table that should be dropped from the Shark metastore and from memory storage. */ def dropTable(databaseName: String, tableName: String): Option[RDD[_]] = { - val tableKey = databaseName + "." + tableName + val tableKey = makeTachyonTableKey(databaseName, tableName) if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(tableKey)) { if (SharkEnv.tachyonUtil.dropTable(tableKey)) { logInfo("Table " + tableKey + " was deleted from Tachyon."); @@ -143,6 +143,14 @@ object SharkEnv extends LogHelper { /** Return the value of an environmental variable as a string. */ def getEnv(varname: String) = if (System.getenv(varname) == null) "" else System.getenv(varname) + /** + * Return an identifier for RDDs that back tables stored in Tachyon. The format is + * "databaseName.tableName". + */ + def makeTachyonTableKey(databaseName: String, tableName: String): String = { + (databaseName + "." + tableName).toLowerCase + } + } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index c6fb1645..d9cc500b 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -139,7 +139,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { } } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. - val tableKey = databaseName + "." + tableName + val tableKey = SharkEnv.makeTachyonTableKey(databaseName, tableName) if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") } From 6ab8a07a6179384bbfa0a896209f155e35d5bd42 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 29 Oct 2013 02:16:59 -0400 Subject: [PATCH 182/331] Removed EmptyRDD given it is in Spark. --- src/main/scala/shark/execution/EmptyRDD.scala | 18 ------------------ .../scala/shark/execution/LimitOperator.scala | 2 +- 2 files changed, 1 insertion(+), 19 deletions(-) delete mode 100644 src/main/scala/shark/execution/EmptyRDD.scala diff --git a/src/main/scala/shark/execution/EmptyRDD.scala b/src/main/scala/shark/execution/EmptyRDD.scala deleted file mode 100644 index 534a34bc..00000000 --- a/src/main/scala/shark/execution/EmptyRDD.scala +++ /dev/null @@ -1,18 +0,0 @@ -package shark.execution - -import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} -import org.apache.spark.rdd.RDD - -/** - * An RDD that is empty, i.e. has no element in it. - * - * TODO: Remove this once EmptyRDD is in Spark. - */ -class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) { - - override def getPartitions: Array[Partition] = Array.empty - - override def compute(split: Partition, context: TaskContext): Iterator[T] = { - throw new UnsupportedOperationException("empty RDD") - } -} diff --git a/src/main/scala/shark/execution/LimitOperator.scala b/src/main/scala/shark/execution/LimitOperator.scala index 2c76fca9..5bd063fd 100755 --- a/src/main/scala/shark/execution/LimitOperator.scala +++ b/src/main/scala/shark/execution/LimitOperator.scala @@ -23,7 +23,7 @@ import scala.reflect.BeanProperty import org.apache.hadoop.hive.ql.exec.{LimitOperator => HiveLimitOperator} import org.apache.hadoop.hive.ql.plan.LimitDesc -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{EmptyRDD, RDD} import shark.SharkEnv From 3083e396e0e51b02cce2a6e9576fe7fd88e1576a Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 19:12:00 -0700 Subject: [PATCH 183/331] Add a HadoopTableReader, for scanning tables stored in Hadoop. --- .../scala/shark/execution/TableReader.scala | 226 ++++++++++++++++ .../shark/execution/TableScanOperator.scala | 242 ++++-------------- 2 files changed, 269 insertions(+), 199 deletions(-) create mode 100644 src/main/scala/shark/execution/TableReader.scala diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala new file mode 100644 index 00000000..2e686692 --- /dev/null +++ b/src/main/scala/shark/execution/TableReader.scala @@ -0,0 +1,226 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution + +import java.io.Serializable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} +import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.JobConf + +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.SerializableWritable + +import shark.{SharkEnv, LogHelper, Utils} + +/** + * A trait for subclasses that handle table scans. In Shark, there is one subclass for each + * type of table storage: HeapTableReader for Shark tables in Spark's block manager, + * TachyonTableReader for tables in Tachyon, and HadoopTableReader for Hive tables in a filesystem. + */ +trait TableReader extends LogHelper{ + + def makeRDDForTable(hiveTable: HiveTable): RDD[_] + + def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] + +} + + +class HeapTableReader() extends TableReader { + + def makeRDDForTable(hiveTable: HiveTable): RDD[_] = null + + def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = null +} + + +class TachyonTableReader() extends TableReader { + + def makeRDDForTable(hiveTable: HiveTable): RDD[_] = null + + def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = null +} + + +class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) + extends TableReader { + + // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless + // it is smaller than what Spark suggests. + val _minSplitsPerRDD = math.max( + _localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) + + HadoopTableReader.addCredentialsToConf(_localHConf) + val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) + + /** + * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed + * RDD that contains deserialized rows. + */ + def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { + assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, + since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") + + // Create local references to member variables, so that the entire `this` object won't be + // serialized in the closure below. + val tableDesc = _tableDesc + val broadcastedHiveConf = _broadcastedHiveConf + + val tablePath = hiveTable.getPath.toString + logDebug("Table input: %s".format(tablePath)) + val ifc = hiveTable.getInputFormatClass + .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] + val hadoopRDD = createHadoopRdd(tableDesc, tablePath, ifc) + + val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => + val hconf = broadcastedHiveConf.value.value + val deserializer = tableDesc.getDeserializerClass().newInstance() + deserializer.initialize(hconf, tableDesc.getProperties) + + // Deserialize each Writable to get the row value. + iter.map { value => + value match { + case v: Writable => deserializer.deserialize(v) + case _ => throw new RuntimeException("Failed to match " + value.toString) + } + } + } + deserializedHadoopRDD + } + + /** + * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive + * tables, a data directory is created for each partition corresponding to keys specified using + * 'PARTITION BY'. + */ + def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = { + val hivePartitionRDDs = partitions.map { partition => + val partDesc = Utilities.getPartitionDesc(partition) + val partPath = partition.getPartitionPath.toString + val ifc = partDesc.getInputFileFormatClass + .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] + // Get partition field info + val partSpec = partDesc.getPartSpec() + val partProps = partDesc.getProperties() + val partDeserializer = partDesc.getDeserializerClass() + + val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + // Partitioning columns are delimited by "/" + val partCols = partColsDelimited.trim().split("/").toSeq + // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. + val partValues = if (partSpec == null) { + Array.fill(partCols.size)(new String) + } else { + partCols.map(col => new String(partSpec.get(col))).toArray + } + + // Create local references so that the outer object isn't serialized. + val tableDesc = _tableDesc + val broadcastedHiveConf = _broadcastedHiveConf + + val hivePartitionRDD = createHadoopRdd(tableDesc, partPath, ifc) + hivePartitionRDD.mapPartitions { iter => + val hconf = broadcastedHiveConf.value.value + val rowWithPartArr = new Array[Object](2) + // Map each tuple to a row object + iter.map { value => + val deserializer = partDeserializer.newInstance() + deserializer.initialize(hconf, partProps) + val deserializedRow = deserializer.deserialize(value) // LazyStruct + rowWithPartArr.update(0, deserializedRow) + rowWithPartArr.update(1, partValues) + rowWithPartArr.asInstanceOf[Object] + } + } + } + // Even if we don't use any partitions, we still need an empty RDD + if (hivePartitionRDDs.size == 0) { + new EmptyRDD[Object](SharkEnv.sc) + } else { + new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) + } + } + + /** + * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be + * applied locally on each slave. + */ + protected def createHadoopRdd( + tableDesc: TableDesc, + path: String, + inputFormatClass: Class[InputFormat[Writable, Writable]]) + : RDD[Writable] = { + val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ + + val rdd = new HadoopRDD( + SharkEnv.sc, + _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], + Some(initializeJobConfFunc), + inputFormatClass, + classOf[Writable], + classOf[Writable], + _minSplitsPerRDD) + + // Only take the value (skip the key) because Hive works only with values. + rdd.map(_._2) + } + +} + +object HadoopTableReader { + + /** + * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to + * instantiate a HadoopRDD. + */ + def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { + FileInputFormat.setInputPaths(jobConf, path) + if (tableDesc != null) { + Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) + } + val bufferSize = System.getProperty("spark.buffer.size", "65536") + jobConf.set("io.file.buffer.size", bufferSize) + } + + /** Adds S3 credentials to the `conf`. */ + def addCredentialsToConf(conf: Configuration) { + // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate + // from Spark's master all the way to Spark's slaves. + var s3varsSet = false + val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", + "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => + if (conf.get(variableName) != null) { + s3varsSet = true + } + } + + // If none of the s3 credentials are set in Hive conf, try use the environmental + // variables for credentials. + if (!s3varsSet) { + Utils.setAwsCredentials(conf) + } + } +} diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index d9cc500b..498f81a5 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -172,43 +172,43 @@ class TableScanOperator extends TopOperator[TableScanDesc] { val columnsUsed = new ColumnPruner(this, table).columnsUsed SharkEnv.tachyonUtil.pushDownColumnPruning(rdd, columnsUsed) - val prunedRdd: RDD[_] = - if (SharkConfVars.getBoolVar(localHConf, SharkConfVars.MAP_PRUNING) && - childOperators(0).isInstanceOf[FilterOperator] && - indexToStats.size == rdd.partitions.size) { - - val startTime = System.currentTimeMillis - val printPruneDebug = SharkConfVars.getBoolVar( - localHConf, SharkConfVars.MAP_PRUNING_PRINT_DEBUG) - - // Must initialize the condition evaluator in FilterOperator to get the - // udfs and object inspectors set. - val filterOp = childOperators(0).asInstanceOf[FilterOperator] - filterOp.initializeOnSlave() - - def prunePartitionFunc(index: Int): Boolean = { - if (printPruneDebug) { - logInfo("\nPartition " + index + "\n" + indexToStats(index)) - } - // Only test for pruning if we have stats on the column. - val partitionStats = indexToStats(index) - if (partitionStats != null && partitionStats.stats != null) { - MapSplitPruning.test(partitionStats, filterOp.conditionEvaluator) - } else { - true - } + val shouldPrune = (SharkConfVars.getBoolVar(localHConf, SharkConfVars.MAP_PRUNING) && + childOperators(0).isInstanceOf[FilterOperator] && + indexToStats.size == rdd.partitions.size) + + val prunedRdd: RDD[_] = if (shouldPrune) { + val startTime = System.currentTimeMillis + val printPruneDebug = SharkConfVars.getBoolVar( + localHConf, SharkConfVars.MAP_PRUNING_PRINT_DEBUG) + + // Must initialize the condition evaluator in FilterOperator to get the + // udfs and object inspectors set. + val filterOp = childOperators(0).asInstanceOf[FilterOperator] + filterOp.initializeOnSlave() + + def prunePartitionFunc(index: Int): Boolean = { + if (printPruneDebug) { + logInfo("\nPartition " + index + "\n" + indexToStats(index)) + } + // Only test for pruning if we have stats on the column. + val partitionStats = indexToStats(index) + if (partitionStats != null && partitionStats.stats != null) { + MapSplitPruning.test(partitionStats, filterOp.conditionEvaluator) + } else { + true } - - // Do the pruning. - val prunedRdd = PartitionPruningRDD.create(rdd, prunePartitionFunc) - val timeTaken = System.currentTimeMillis - startTime - logInfo("Map pruning %d partitions into %s partitions took %d ms".format( - rdd.partitions.size, prunedRdd.partitions.size, timeTaken)) - prunedRdd - } else { - rdd } + // Do the pruning. + val prunedRdd = PartitionPruningRDD.create(rdd, prunePartitionFunc) + val timeTaken = System.currentTimeMillis - startTime + logInfo("Map pruning %d partitions into %s partitions took %d ms".format( + rdd.partitions.size, prunedRdd.partitions.size, timeTaken)) + prunedRdd + } else { + rdd + } + return prunedRdd.mapPartitions { iter => if (iter.hasNext) { val tablePartition = iter.next.asInstanceOf[TablePartition] @@ -221,55 +221,22 @@ class TableScanOperator extends TopOperator[TableScanDesc] { } /** - * Create a RDD for a table. + * Create an RDD for a table stored in Hadoop. */ def makeRDDFromHadoop(): RDD[_] = { - // Choose the minimum number of splits. If mapred.map.tasks is set, use that unless - // it is smaller than what Spark suggests. - val minSplitsPerRDD = math.max( - localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) - - TableScanOperator.prepareHiveConf(localHConf, hiveOp) - val broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(localHConf)) + // Try to have the InputFormats filter predicates. + TableScanOperator.addFilterExprToConf(localHConf, hiveOp) + val hadoopReader = new HadoopTableReader(tableDesc, localHConf) if (table.isPartitioned) { logDebug("Making %d Hive partitions".format(parts.size)) // The returned RDD contains arrays of size two with the elements as // (deserialized row, column partition value). - return makeHivePartitionRDDs(broadcastedHiveConf, minSplitsPerRDD) + return hadoopReader.makeRDDForTablePartitions(parts) } else { // The returned RDD contains deserialized row Objects. - return makeTableRDD(broadcastedHiveConf, minSplitsPerRDD) - } - } - - /** - * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed - * RDD that contains deserialized rows. - */ - private def makeTableRDD( - broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], - minSplits: Int): RDD[_] = { - val tablePath = table.getPath.toString - val ifc = table.getInputFormatClass - .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - logDebug("Table input: %s".format(tablePath)) - - val hadoopRDD = createHadoopRdd(tablePath, ifc, broadcastedHiveConf, minSplits) - val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => - val hconf = broadcastedHiveConf.value.value - val deserializer = tableDesc.getDeserializerClass().newInstance() - deserializer.initialize(hconf, tableDesc.getProperties) - - // Deserialize each Writable to get the row value. - iter.map { value => - value match { - case v: Writable => deserializer.deserialize(v) - case _ => throw new RuntimeException("Failed to match " + value.toString) - } - } + return hadoopReader.makeRDDForTable(table) } - return deserializedHadoopRDD } /** @@ -327,105 +294,6 @@ class TableScanOperator extends TopOperator[TableScanDesc] { } } - /** - * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive - * tables, a data directory is created for each partition corresponding to keys specified using - * 'PARTITION BY'. - */ - private def makeHivePartitionRDDs[T]( - broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], - minSplitsPerRDD: Int - ): RDD[_] = { - val partitions = parts - val hivePartitionRDDs = new Array[RDD[Any]](partitions.size) - - var i = 0 - partitions.foreach { partition => - val partDesc = Utilities.getPartitionDesc(partition) - val tablePath = partition.getPartitionPath.toString - - val ifc = partition.getInputFormatClass - .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - val hivePartitionRDD = createHadoopRdd( - tablePath, ifc, broadcastedHiveConf, minSplitsPerRDD) - - val hivePartitionRDDWithColValues = hivePartitionRDD.mapPartitions { iter => - val hconf = broadcastedHiveConf.value.value - val deserializer = partDesc.getDeserializerClass().newInstance() - deserializer.initialize(hconf, partDesc.getProperties()) - - // Get partition field info - val partSpec = partDesc.getPartSpec() - val partProps = partDesc.getProperties() - - val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - // Partitioning keys are delimited by "/" - val partCols = partColsDelimited.trim().split("/") - // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. - val partValues = if (partSpec == null) { - Array.fill(partCols.size)(new String) - } else { - partCols.map(col => new String(partSpec.get(col))).toArray - } - - val rowWithPartArr = new Array[Object](2) - // Map each tuple to a row object - iter.map { value => - val deserializedRow = deserializer.deserialize(value) // LazyStruct - rowWithPartArr.update(0, deserializedRow) - rowWithPartArr.update(1, partValues) - rowWithPartArr.asInstanceOf[Object] - } - } - hivePartitionRDDs(i) = hivePartitionRDDWithColValues.asInstanceOf[RDD[Any]] - i += 1 - } - // Even if we don't use any partitions, we still need an empty RDD - if (hivePartitionRDDs.size == 0) { - SharkEnv.sc.makeRDD(Seq[Object]()) - } else { - new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) - } - } - - /** - * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be - * applied locally on each slave. - */ - private def createHadoopRdd( - path: String, - inputFormatClass: Class[InputFormat[Writable, Writable]], - broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], - minSplits: Int) - : RDD[Writable] = { - /* - * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to - * instantiate a HadoopRDD. - */ - def initializeLocalJobConfFunc(path: String)(jobConf: JobConf) { - FileInputFormat.setInputPaths(jobConf, path) - if (tableDesc != null) { - Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) - } - val bufferSize = System.getProperty("spark.buffer.size", "65536") - jobConf.set("io.file.buffer.size", bufferSize) - } - - val initializeJobConfFunc = initializeLocalJobConfFunc(path) _ - - val rdd = new HadoopRDD( - SharkEnv.sc, - broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], - Some(initializeJobConfFunc), - inputFormatClass, - classOf[Writable], - classOf[Writable], - minSplits) - - // Only take the value (skip the key) because Hive works only with values. - rdd.map(_._2) - } - // All RDD processing is done in execute(). override def processPartition(split: Int, iter: Iterator[_]): Iterator[_] = throw new UnsupportedOperationException("TableScanOperator.processPartition()") @@ -435,33 +303,9 @@ class TableScanOperator extends TopOperator[TableScanDesc] { object TableScanOperator extends LogHelper { /** - * Add miscellaneous properties to the HiveConf to be used for creating a HadoopRDD. These - * properties are impractical to add during local JobConf creation in HadoopRDD - for example, - * filter expressions would require a serialized HiveTableScanOperator. - */ - private def prepareHiveConf(hiveConf: HiveConf, hiveTableScanOp: HiveTableScanOperator) { - // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate - // from Spark's master all the way to Spark's slaves. - var s3varsSet = false - val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", - "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => - if (hiveConf.get(variableName) != null) { - s3varsSet = true - } - } - - // If none of the s3 credentials are set in Hive conf, try use the environmental - // variables for credentials. - if (!s3varsSet) { - Utils.setAwsCredentials(hiveConf) - } - - TableScanOperator.addFilterExprToConf(hiveConf, hiveTableScanOp) - } - - /** - * Add filter expressions and column metadata to the HiveConf This is meant to be called on the - * master, so that we can avoid serializing the HiveTableScanOperator. + * Add filter expressions and column metadata to the HiveConf. This is meant to be called on the + * master - it's impractical to add filters during slave-local JobConf creation in HadoopRDD, + * since we would have to serialize the HiveTableScanOperator. */ private def addFilterExprToConf(hiveConf: HiveConf, hiveTableScanOp: HiveTableScanOperator) { val tableScanDesc = hiveTableScanOp.getConf() From 891334039117978a6587aac8c91664d0c0a03cd6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 22:04:28 -0700 Subject: [PATCH 184/331] Add a HeapTableReader, for scanning tables stored in the Spark block manager. --- .../scala/shark/execution/TableReader.scala | 142 ++++++++++++++++-- .../shark/execution/TableScanOperator.scala | 77 +--------- 2 files changed, 135 insertions(+), 84 deletions(-) diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 2e686692..d313fe53 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -20,6 +20,7 @@ package shark.execution import java.io.Serializable import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS @@ -33,7 +34,11 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.SerializableWritable -import shark.{SharkEnv, LogHelper, Utils} +import shark.api.QueryExecutionException +import shark.execution.optimization.ColumnPruner +import shark.{LogHelper, SharkConfVars, SharkEnv, Utils} +import shark.memstore2.{MemoryMetadataManager, TablePartition, TablePartitionStats} + /** * A trait for subclasses that handle table scans. In Shark, there is one subclass for each @@ -49,19 +54,102 @@ trait TableReader extends LogHelper{ } -class HeapTableReader() extends TableReader { +class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { + + // Split from 'databaseName.tableName' + private val _tableNameSplit = _tableDesc.getTableName.split('.') + private val _databaseName = _tableNameSplit(0) + private val _tableName = _tableNameSplit(1) + + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { + logInfo("Loading table %s.%s from Spark block manager".format(_databaseName, _tableName)) + val tableOpt = SharkEnv.memoryMetadataManager.getMemoryTable(_databaseName, _tableName) + if (tableOpt.isEmpty) throwMissingTableException() + + val table = tableOpt.get + table.tableRDD + } + + /** + * Fetch an RDD from the Shark metastore using each partition key given, and return a union of all + * the fetched RDDs. + * + * @param tableKey Name of the partitioned table. + * @param partitions A collection of Hive-partition metadata, such as partition columns and + * partition key specifications. + */ + override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = { + val hivePartitionRDDs = partitions.map { partition => + val partDesc = Utilities.getPartitionDesc(partition) + // Get partition field info + val partSpec = partDesc.getPartSpec() + val partProps = partDesc.getProperties() + + val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + // Partitioning columns are delimited by "/" + val partCols = partColsDelimited.trim().split("/").toSeq + // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. + val partValues = if (partSpec == null) { + Array.fill(partCols.size)(new String) + } else { + partCols.map(col => new String(partSpec.get(col))).toArray + } + + val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) + val hivePartitionedTableOpt = SharkEnv.memoryMetadataManager.getPartitionedTable( + _databaseName, _tableName) + if (hivePartitionedTableOpt.isEmpty) throwMissingTableException() + val hivePartitionedTable = hivePartitionedTableOpt.get + + val hivePartitionRDDOpt = hivePartitionedTable.getPartition(partitionKeyStr) + if (hivePartitionRDDOpt.isEmpty) throwMissingPartitionException(partitionKeyStr) + val hivePartitionRDD = hivePartitionRDDOpt.get + + hivePartitionRDD.mapPartitions { iter => + if (iter.hasNext) { + // Map each tuple to a row object + val rowWithPartArr = new Array[Object](2) + val tablePartition = iter.next.asInstanceOf[TablePartition] + tablePartition.iterator.map { value => + rowWithPartArr.update(0, value.asInstanceOf[Object]) + rowWithPartArr.update(1, partValues) + rowWithPartArr.asInstanceOf[Object] + } + } else { + Iterator() + } + } + } + if (hivePartitionRDDs.size > 0) { + new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs) + } else { + new EmptyRDD[Object](SharkEnv.sc) + } + } + + private def throwMissingTableException() { + logError("""|Table %s.%s not found in block manager. + |Are you trying to access a cached table from a Shark session other than the one + |in which it was created?""".stripMargin.format(_databaseName, _tableName)) + throw new QueryExecutionException("Cached table not found") + } - def makeRDDForTable(hiveTable: HiveTable): RDD[_] = null + private def throwMissingPartitionException(partValues: String) { + logError("""|Partition %s for table %s.%s not found in block manager. + |Are you trying to access a cached table from a Shark session other than the one in + |which it was created?""".stripMargin.format(partValues, _databaseName, _tableName)) + throw new QueryExecutionException("Cached table partition not found") + } - def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = null } class TachyonTableReader() extends TableReader { - def makeRDDForTable(hiveTable: HiveTable): RDD[_] = null + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = null + + override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = null - def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = null } @@ -70,17 +158,22 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless // it is smaller than what Spark suggests. - val _minSplitsPerRDD = math.max( + private val _minSplitsPerRDD = math.max( _localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) HadoopTableReader.addCredentialsToConf(_localHConf) - val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) + private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) + + def broadcastedHiveConf = _broadcastedHiveConf + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = + makeRDDForTable(hiveTable, filterOpt = None) + /** * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed * RDD that contains deserialized rows. */ - def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { + def makeRDDForTable(hiveTable: HiveTable, filterOpt: Option[PathFilter] = None): RDD[_] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") @@ -89,11 +182,13 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val tableDesc = _tableDesc val broadcastedHiveConf = _broadcastedHiveConf - val tablePath = hiveTable.getPath.toString + val tablePath = hiveTable.getPath + val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) + logDebug("Table input: %s".format(tablePath)) val ifc = hiveTable.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - val hadoopRDD = createHadoopRdd(tableDesc, tablePath, ifc) + val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value @@ -111,15 +206,21 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf deserializedHadoopRDD } + override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = + makeRDDForTablePartitions(partitions, filterOpt = None) + /** * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive * tables, a data directory is created for each partition corresponding to keys specified using * 'PARTITION BY'. */ - def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = { + def makeRDDForTablePartitions( + partitions: Seq[Partition], + filterOpt: Option[PathFilter]): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) - val partPath = partition.getPartitionPath.toString + val partPath = partition.getPartitionPath + val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) val ifc = partDesc.getInputFileFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] // Get partition field info @@ -141,7 +242,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val tableDesc = _tableDesc val broadcastedHiveConf = _broadcastedHiveConf - val hivePartitionRDD = createHadoopRdd(tableDesc, partPath, ifc) + val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) hivePartitionRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val rowWithPartArr = new Array[Object](2) @@ -164,11 +265,22 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf } } + private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { + filterOpt match { + case Some(filter) => { + val fs = path.getFileSystem(_localHConf) + val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) + filteredFiles.mkString(",") + } + case None => path.toString + } + } + /** * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be * applied locally on each slave. */ - protected def createHadoopRdd( + private def createHadoopRdd( tableDesc: TableDesc, path: String, inputFormatClass: Class[InputFormat[Writable, Writable]]) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 498f81a5..6f546e51 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -114,6 +114,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { val tableNameSplit = tableDesc.getTableName.split('.') // Split from 'databaseName.tableName' val databaseName = tableNameSplit(0) val tableName = tableNameSplit(1) + // There are three places we can load the table from. // 1. Tachyon table // 2. Spark heap (block manager), accessed through the Shark MemoryMetadataManager @@ -122,20 +123,12 @@ class TableScanOperator extends TopOperator[TableScanDesc] { tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. if (cacheMode == CacheType.HEAP) { - // Table should be in Spark heap (block manager). - if (!SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { - logError("""|Table %s.%s not found in block manager. - |Are you trying to access a cached table from a Shark session other than the one - |in which it was created?""".stripMargin.format(databaseName, tableName)) - throw new QueryExecutionException("Cached table not found") - } - if (SharkEnv.memoryMetadataManager.isHivePartitioned(databaseName, tableName)) { - // Get the union of RDDs repesenting the selected Hive partition(s). - return makeCachedPartitionRDD(databaseName, tableName, parts) + val tableReader = new HeapTableReader(tableDesc) + if (table.isPartitioned) { + return tableReader.makeRDDForTablePartitions(parts) } else { - val table = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName).get - logInfo("Loading table %s.%s from Spark block manager".format(databaseName, tableName)) - return createPrunedRdd(databaseName, tableName, table.tableRDD) + val tableRdd = tableReader.makeRDDForTable(table) + return createPrunedRdd(databaseName, tableName, tableRdd) } } else if (cacheMode == CacheType.TACHYON) { // Table is in Tachyon. @@ -213,7 +206,6 @@ class TableScanOperator extends TopOperator[TableScanDesc] { if (iter.hasNext) { val tablePartition = iter.next.asInstanceOf[TablePartition] tablePartition.prunedIterator(columnsUsed) - //tablePartition.iterator } else { Iterator() } @@ -239,64 +231,10 @@ class TableScanOperator extends TopOperator[TableScanDesc] { } } - /** - * Fetch an RDD from the Shark metastore using each partition key given, and return a union of all - * the fetched RDDs. - * - * @param tableKey Name of the partitioned table. - * @param partitions A collection of Hive-partition metadata, such as partition columns and - * partition key specifications. - */ - private def makeCachedPartitionRDD( - databaseName: String, - tableName: String, - partitions: Array[Partition]): RDD[_] = { - val hivePartitionRDDs = partitions.map { partition => - val partDesc = Utilities.getPartitionDesc(partition) - // Get partition field info - val partSpec = partDesc.getPartSpec() - val partProps = partDesc.getProperties() - - val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - // Partitioning columns are delimited by "/" - val partCols = partColsDelimited.trim().split("/").toSeq - // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. - val partValues = if (partSpec == null) { - Array.fill(partCols.size)(new String) - } else { - partCols.map(col => new String(partSpec.get(col))).toArray - } - - val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) - val hivePartitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - databaseName, tableName).get - val hivePartitionRDD = hivePartitionedTable.getPartition(partitionKeyStr) - - hivePartitionRDD.get.mapPartitions { iter => - if (iter.hasNext) { - // Map each tuple to a row object - val rowWithPartArr = new Array[Object](2) - val tablePartition = iter.next.asInstanceOf[TablePartition] - tablePartition.iterator.map { value => - rowWithPartArr.update(0, value.asInstanceOf[Object]) - rowWithPartArr.update(1, partValues) - rowWithPartArr.asInstanceOf[Object] - } - } else { - Iterator() - } - } - } - if (hivePartitionRDDs.size > 0) { - new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs) - } else { - new EmptyRDD[Object](SharkEnv.sc) - } - } - // All RDD processing is done in execute(). override def processPartition(split: Int, iter: Iterator[_]): Iterator[_] = throw new UnsupportedOperationException("TableScanOperator.processPartition()") + } @@ -348,4 +286,5 @@ object TableScanOperator extends LogHelper { logDebug("Filter expression: " + filterExprNodeSerialized) } } + } From db7166755c675726cc65bb2e568758f0ac776330 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 22:36:20 -0700 Subject: [PATCH 185/331] Add a TachyonTableReader, for tables managed by Tachyon. Partitioned tables in Tachyon are not yet supported, though. --- .../scala/shark/execution/TableReader.scala | 45 ++++++++++++++----- 1 file changed, 35 insertions(+), 10 deletions(-) diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index d313fe53..692231a1 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -36,8 +36,10 @@ import org.apache.spark.SerializableWritable import shark.api.QueryExecutionException import shark.execution.optimization.ColumnPruner +import shark.execution.serialization.JavaSerializer import shark.{LogHelper, SharkConfVars, SharkEnv, Utils} import shark.memstore2.{MemoryMetadataManager, TablePartition, TablePartitionStats} +import shark.tachyon.TachyonException /** @@ -53,6 +55,39 @@ trait TableReader extends LogHelper{ } +class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { + + // Split from 'databaseName.tableName' + private val _tableNameSplit = _tableDesc.getTableName.split('.') + private val _databaseName = _tableNameSplit(0) + private val _tableName = _tableNameSplit(1) + + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { + // Table is in Tachyon. + val tableKey = SharkEnv.makeTachyonTableKey(_databaseName, _tableName) + if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { + throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") + } + logInfo("Loading table " + tableKey + " from Tachyon.") + + var indexToStats: collection.Map[Int, TablePartitionStats] = + SharkEnv.memoryMetadataManager.getStats(_databaseName, _tableName).getOrElse(null) + + if (indexToStats == null) { + val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) + indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( + statsByteBuffer.array()) + logInfo("Loading table " + tableKey + " stats from Tachyon.") + SharkEnv.memoryMetadataManager.putStats(_databaseName, _tableName, indexToStats) + } + SharkEnv.tachyonUtil.createRDD(tableKey) + } + + override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = { + throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") + } + +} class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { @@ -143,16 +178,6 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { } - -class TachyonTableReader() extends TableReader { - - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = null - - override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = null - -} - - class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) extends TableReader { From cd2e7d93467a94eaceb3b248856c03c3742da382 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 22:54:33 -0700 Subject: [PATCH 186/331] makeRDDForTablePartitions => makeRDDForPartitionedTable, since the former can be confused with RDD[TablePartition]. --- src/main/scala/shark/execution/TableReader.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 692231a1..f8872062 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -51,7 +51,7 @@ trait TableReader extends LogHelper{ def makeRDDForTable(hiveTable: HiveTable): RDD[_] - def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] + def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] } @@ -83,7 +83,7 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { SharkEnv.tachyonUtil.createRDD(tableKey) } - override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = { + override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = { throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") } @@ -113,7 +113,7 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. */ - override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = { + override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) // Get partition field info @@ -231,15 +231,15 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf deserializedHadoopRDD } - override def makeRDDForTablePartitions(partitions: Seq[Partition]): RDD[_] = - makeRDDForTablePartitions(partitions, filterOpt = None) - + override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = + makeRDDForPartitionedTable(partitions, filterOpt = None) + /** * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive * tables, a data directory is created for each partition corresponding to keys specified using * 'PARTITION BY'. */ - def makeRDDForTablePartitions( + def makeRDDForPartitionedTable( partitions: Seq[Partition], filterOpt: Option[PathFilter]): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => From 130d7b731b78d20f5b9b562dca6e93dd43293ee8 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 23:13:47 -0700 Subject: [PATCH 187/331] Some comments for classes in TableReader.scala. --- .../scala/shark/execution/TableReader.scala | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index f8872062..1150831b 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -55,6 +55,7 @@ trait TableReader extends LogHelper{ } +/** Helper class for scanning tables stored in Tachyon. */ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { // Split from 'databaseName.tableName' @@ -89,6 +90,7 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { } +/** Helper class for scanning tables stored in Spark's block manager */ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { // Split from 'databaseName.tableName' @@ -96,6 +98,7 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { private val _databaseName = _tableNameSplit(0) private val _tableName = _tableNameSplit(1) + /** Fetches the RDD for `_tableName` from the Shark metastore. */ override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { logInfo("Loading table %s.%s from Spark block manager".format(_databaseName, _tableName)) val tableOpt = SharkEnv.memoryMetadataManager.getMemoryTable(_databaseName, _tableName) @@ -109,7 +112,6 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { * Fetch an RDD from the Shark metastore using each partition key given, and return a union of all * the fetched RDDs. * - * @param tableKey Name of the partitioned table. * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. */ @@ -162,6 +164,10 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { } } + /** + * Thrown if the table identified by the (_databaseName, _tableName) pair cannot be found in + * the Shark metastore. + */ private def throwMissingTableException() { logError("""|Table %s.%s not found in block manager. |Are you trying to access a cached table from a Shark session other than the one @@ -169,6 +175,10 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { throw new QueryExecutionException("Cached table not found") } + /** + * Thrown if the table partition identified by the (_databaseName, _tableName, partValues) tuple + * cannot be found in the Shark metastore. + */ private def throwMissingPartitionException(partValues: String) { logError("""|Partition %s for table %s.%s not found in block manager. |Are you trying to access a cached table from a Shark session other than the one in @@ -178,6 +188,10 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { } +/** + * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the + * data warehouse directory. + */ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) extends TableReader { @@ -186,6 +200,8 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf private val _minSplitsPerRDD = math.max( _localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) + // Add security credentials before broadcasting the Hive configuration, which is used accross all + // reads done by an instance of this class. HadoopTableReader.addCredentialsToConf(_localHConf) private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) @@ -290,6 +306,10 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf } } + /** + * If `filterOpt` is defined, then it will be used to filter files from `path`. These files are + * returned in a single, comma-separated string. + */ private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { filterOpt match { case Some(filter) => { From b61576568613d1ee55470d2cebef59717dbd99ec Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 23:54:03 -0700 Subject: [PATCH 188/331] Put HadoopTableReader in its own file. --- .../shark/execution/HadoopTableReader.scala | 229 ++++++++++++++++++ .../scala/shark/execution/TableReader.scala | 205 +--------------- 2 files changed, 232 insertions(+), 202 deletions(-) create mode 100644 src/main/scala/shark/execution/HadoopTableReader.scala diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala new file mode 100644 index 00000000..e73a79cd --- /dev/null +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -0,0 +1,229 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} +import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} +import org.apache.hadoop.io.Writable + +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.SerializableWritable + +import shark.{SharkEnv, Utils} + + +/** + * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the + * data warehouse directory. + */ +class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) + extends TableReader { + + // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless + // it is smaller than what Spark suggests. + private val _minSplitsPerRDD = math.max( + _localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) + + // Add security credentials before broadcasting the Hive configuration, which is used accross all + // reads done by an instance of this class. + HadoopTableReader.addCredentialsToConf(_localHConf) + private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) + + def broadcastedHiveConf = _broadcastedHiveConf + + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = + makeRDDForTable(hiveTable, filterOpt = None) + + /** + * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed + * RDD that contains deserialized rows. + */ + def makeRDDForTable(hiveTable: HiveTable, filterOpt: Option[PathFilter] = None): RDD[_] = { + assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, + since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") + + // Create local references to member variables, so that the entire `this` object won't be + // serialized in the closure below. + val tableDesc = _tableDesc + val broadcastedHiveConf = _broadcastedHiveConf + + val tablePath = hiveTable.getPath + val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) + + logDebug("Table input: %s".format(tablePath)) + val ifc = hiveTable.getInputFormatClass + .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] + val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) + + val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => + val hconf = broadcastedHiveConf.value.value + val deserializer = tableDesc.getDeserializerClass().newInstance() + deserializer.initialize(hconf, tableDesc.getProperties) + + // Deserialize each Writable to get the row value. + iter.map { value => + value match { + case v: Writable => deserializer.deserialize(v) + case _ => throw new RuntimeException("Failed to match " + value.toString) + } + } + } + deserializedHadoopRDD + } + + override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = + makeRDDForPartitionedTable(partitions, filterOpt = None) + + /** + * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive + * tables, a data directory is created for each partition corresponding to keys specified using + * 'PARTITION BY'. + */ + def makeRDDForPartitionedTable( + partitions: Seq[Partition], + filterOpt: Option[PathFilter]): RDD[_] = { + val hivePartitionRDDs = partitions.map { partition => + val partDesc = Utilities.getPartitionDesc(partition) + val partPath = partition.getPartitionPath + val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) + val ifc = partDesc.getInputFileFormatClass + .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] + // Get partition field info + val partSpec = partDesc.getPartSpec() + val partProps = partDesc.getProperties() + val partDeserializer = partDesc.getDeserializerClass() + + val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + // Partitioning columns are delimited by "/" + val partCols = partColsDelimited.trim().split("/").toSeq + // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. + val partValues = if (partSpec == null) { + Array.fill(partCols.size)(new String) + } else { + partCols.map(col => new String(partSpec.get(col))).toArray + } + + // Create local references so that the outer object isn't serialized. + val tableDesc = _tableDesc + val broadcastedHiveConf = _broadcastedHiveConf + + val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) + hivePartitionRDD.mapPartitions { iter => + val hconf = broadcastedHiveConf.value.value + val rowWithPartArr = new Array[Object](2) + // Map each tuple to a row object + iter.map { value => + val deserializer = partDeserializer.newInstance() + deserializer.initialize(hconf, partProps) + val deserializedRow = deserializer.deserialize(value) // LazyStruct + rowWithPartArr.update(0, deserializedRow) + rowWithPartArr.update(1, partValues) + rowWithPartArr.asInstanceOf[Object] + } + } + } + // Even if we don't use any partitions, we still need an empty RDD + if (hivePartitionRDDs.size == 0) { + new EmptyRDD[Object](SharkEnv.sc) + } else { + new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) + } + } + + /** + * If `filterOpt` is defined, then it will be used to filter files from `path`. These files are + * returned in a single, comma-separated string. + */ + private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { + filterOpt match { + case Some(filter) => { + val fs = path.getFileSystem(_localHConf) + val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) + filteredFiles.mkString(",") + } + case None => path.toString + } + } + + /** + * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be + * applied locally on each slave. + */ + private def createHadoopRdd( + tableDesc: TableDesc, + path: String, + inputFormatClass: Class[InputFormat[Writable, Writable]]) + : RDD[Writable] = { + val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ + + val rdd = new HadoopRDD( + SharkEnv.sc, + _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], + Some(initializeJobConfFunc), + inputFormatClass, + classOf[Writable], + classOf[Writable], + _minSplitsPerRDD) + + // Only take the value (skip the key) because Hive works only with values. + rdd.map(_._2) + } + +} + +object HadoopTableReader { + + /** + * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to + * instantiate a HadoopRDD. + */ + def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { + FileInputFormat.setInputPaths(jobConf, path) + if (tableDesc != null) { + Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) + } + val bufferSize = System.getProperty("spark.buffer.size", "65536") + jobConf.set("io.file.buffer.size", bufferSize) + } + + /** Adds S3 credentials to the `conf`. */ + def addCredentialsToConf(conf: Configuration) { + // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate + // from Spark's master all the way to Spark's slaves. + var s3varsSet = false + val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", + "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => + if (conf.get(variableName) != null) { + s3varsSet = true + } + } + + // If none of the s3 credentials are set in Hive conf, try use the environmental + // variables for credentials. + if (!s3varsSet) { + Utils.setAwsCredentials(conf) + } + } +} \ No newline at end of file diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 1150831b..32ed3c0e 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -17,27 +17,22 @@ package shark.execution -import java.io.Serializable - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} -import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} +import org.apache.hadoop.fs.PathFilter import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred.JobConf -import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.SerializableWritable import shark.api.QueryExecutionException import shark.execution.optimization.ColumnPruner import shark.execution.serialization.JavaSerializer -import shark.{LogHelper, SharkConfVars, SharkEnv, Utils} +import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.memstore2.{MemoryMetadataManager, TablePartition, TablePartitionStats} import shark.tachyon.TachyonException @@ -187,197 +182,3 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { } } - -/** - * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the - * data warehouse directory. - */ -class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) - extends TableReader { - - // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless - // it is smaller than what Spark suggests. - private val _minSplitsPerRDD = math.max( - _localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) - - // Add security credentials before broadcasting the Hive configuration, which is used accross all - // reads done by an instance of this class. - HadoopTableReader.addCredentialsToConf(_localHConf) - private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) - - def broadcastedHiveConf = _broadcastedHiveConf - - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = - makeRDDForTable(hiveTable, filterOpt = None) - - /** - * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed - * RDD that contains deserialized rows. - */ - def makeRDDForTable(hiveTable: HiveTable, filterOpt: Option[PathFilter] = None): RDD[_] = { - assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, - since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") - - // Create local references to member variables, so that the entire `this` object won't be - // serialized in the closure below. - val tableDesc = _tableDesc - val broadcastedHiveConf = _broadcastedHiveConf - - val tablePath = hiveTable.getPath - val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) - - logDebug("Table input: %s".format(tablePath)) - val ifc = hiveTable.getInputFormatClass - .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) - - val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => - val hconf = broadcastedHiveConf.value.value - val deserializer = tableDesc.getDeserializerClass().newInstance() - deserializer.initialize(hconf, tableDesc.getProperties) - - // Deserialize each Writable to get the row value. - iter.map { value => - value match { - case v: Writable => deserializer.deserialize(v) - case _ => throw new RuntimeException("Failed to match " + value.toString) - } - } - } - deserializedHadoopRDD - } - - override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = - makeRDDForPartitionedTable(partitions, filterOpt = None) - - /** - * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive - * tables, a data directory is created for each partition corresponding to keys specified using - * 'PARTITION BY'. - */ - def makeRDDForPartitionedTable( - partitions: Seq[Partition], - filterOpt: Option[PathFilter]): RDD[_] = { - val hivePartitionRDDs = partitions.map { partition => - val partDesc = Utilities.getPartitionDesc(partition) - val partPath = partition.getPartitionPath - val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) - val ifc = partDesc.getInputFileFormatClass - .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - // Get partition field info - val partSpec = partDesc.getPartSpec() - val partProps = partDesc.getProperties() - val partDeserializer = partDesc.getDeserializerClass() - - val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - // Partitioning columns are delimited by "/" - val partCols = partColsDelimited.trim().split("/").toSeq - // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. - val partValues = if (partSpec == null) { - Array.fill(partCols.size)(new String) - } else { - partCols.map(col => new String(partSpec.get(col))).toArray - } - - // Create local references so that the outer object isn't serialized. - val tableDesc = _tableDesc - val broadcastedHiveConf = _broadcastedHiveConf - - val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) - hivePartitionRDD.mapPartitions { iter => - val hconf = broadcastedHiveConf.value.value - val rowWithPartArr = new Array[Object](2) - // Map each tuple to a row object - iter.map { value => - val deserializer = partDeserializer.newInstance() - deserializer.initialize(hconf, partProps) - val deserializedRow = deserializer.deserialize(value) // LazyStruct - rowWithPartArr.update(0, deserializedRow) - rowWithPartArr.update(1, partValues) - rowWithPartArr.asInstanceOf[Object] - } - } - } - // Even if we don't use any partitions, we still need an empty RDD - if (hivePartitionRDDs.size == 0) { - new EmptyRDD[Object](SharkEnv.sc) - } else { - new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) - } - } - - /** - * If `filterOpt` is defined, then it will be used to filter files from `path`. These files are - * returned in a single, comma-separated string. - */ - private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { - filterOpt match { - case Some(filter) => { - val fs = path.getFileSystem(_localHConf) - val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) - filteredFiles.mkString(",") - } - case None => path.toString - } - } - - /** - * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be - * applied locally on each slave. - */ - private def createHadoopRdd( - tableDesc: TableDesc, - path: String, - inputFormatClass: Class[InputFormat[Writable, Writable]]) - : RDD[Writable] = { - val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ - - val rdd = new HadoopRDD( - SharkEnv.sc, - _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], - Some(initializeJobConfFunc), - inputFormatClass, - classOf[Writable], - classOf[Writable], - _minSplitsPerRDD) - - // Only take the value (skip the key) because Hive works only with values. - rdd.map(_._2) - } - -} - -object HadoopTableReader { - - /** - * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to - * instantiate a HadoopRDD. - */ - def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { - FileInputFormat.setInputPaths(jobConf, path) - if (tableDesc != null) { - Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) - } - val bufferSize = System.getProperty("spark.buffer.size", "65536") - jobConf.set("io.file.buffer.size", bufferSize) - } - - /** Adds S3 credentials to the `conf`. */ - def addCredentialsToConf(conf: Configuration) { - // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate - // from Spark's master all the way to Spark's slaves. - var s3varsSet = false - val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", - "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => - if (conf.get(variableName) != null) { - s3varsSet = true - } - } - - // If none of the s3 credentials are set in Hive conf, try use the environmental - // variables for credentials. - if (!s3varsSet) { - Utils.setAwsCredentials(conf) - } - } -} From 2e4a14e418c3e9ee4a07323123691f09fd2df707 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 23:56:50 -0700 Subject: [PATCH 189/331] Cleanup TableScanOperator imports. This also gets it to compile... --- .../shark/execution/HadoopTableReader.scala | 3 +- .../scala/shark/execution/TableReader.scala | 2 - .../shark/execution/TableScanOperator.scala | 42 +++++-------------- 3 files changed, 12 insertions(+), 35 deletions(-) diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index e73a79cd..0547aa46 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -190,7 +190,6 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) } - } object HadoopTableReader { @@ -226,4 +225,4 @@ object HadoopTableReader { Utils.setAwsCredentials(conf) } } -} \ No newline at end of file +} diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 32ed3c0e..3791a2cb 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -82,7 +82,6 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = { throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") } - } /** Helper class for scanning tables stored in Spark's block manager */ @@ -180,5 +179,4 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { |which it was created?""".stripMargin.format(partValues, _databaseName, _tableName)) throw new QueryExecutionException("Cached table partition not found") } - } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 6f546e51..bca3ddc8 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -22,31 +22,22 @@ import java.util.{ArrayList, Arrays} import scala.collection.JavaConversions._ import scala.reflect.BeanProperty -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.ql.exec.{TableScanOperator => HiveTableScanOperator} import org.apache.hadoop.hive.ql.exec.{MapSplitPruning, Utilities} -import org.apache.hadoop.hive.ql.io.HiveInputFormat import org.apache.hadoop.hive.ql.metadata.{Partition, Table} -import org.apache.hadoop.hive.ql.plan.{PlanUtils, PartitionDesc, TableDesc, TableScanDesc} +import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc, TableScanDesc} import org.apache.hadoop.hive.serde.Constants import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.hadoop.io.Writable -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, PartitionPruningRDD, RDD, UnionRDD} -import org.apache.spark.SerializableWritable +import org.apache.spark.rdd.{PartitionPruningRDD, RDD} -import shark.{LogHelper, SharkConfVars, SharkEnv, Utils} -import shark.api.QueryExecutionException +import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.optimization.ColumnPruner -import shark.execution.serialization.{XmlSerializer, JavaSerializer} import shark.memstore2.{CacheType, MemoryMetadataManager, TablePartition, TablePartitionStats} -import shark.tachyon.TachyonException /** @@ -125,30 +116,19 @@ class TableScanOperator extends TopOperator[TableScanDesc] { if (cacheMode == CacheType.HEAP) { val tableReader = new HeapTableReader(tableDesc) if (table.isPartitioned) { - return tableReader.makeRDDForTablePartitions(parts) + return tableReader.makeRDDForPartitionedTable(parts) } else { val tableRdd = tableReader.makeRDDForTable(table) return createPrunedRdd(databaseName, tableName, tableRdd) } } else if (cacheMode == CacheType.TACHYON) { - // Table is in Tachyon. - val tableKey = SharkEnv.makeTachyonTableKey(databaseName, tableName) - if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { - throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") - } - logInfo("Loading table " + tableKey + " from Tachyon.") - - var indexToStats: collection.Map[Int, TablePartitionStats] = - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName).getOrElse(null) - - if (indexToStats == null) { - val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) - indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( - statsByteBuffer.array()) - logInfo("Loading table " + tableKey + " stats from Tachyon.") - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, indexToStats) + val tableReader = new TachyonTableReader(tableDesc) + if (table.isPartitioned) { + return tableReader.makeRDDForPartitionedTable(parts) + } else { + val tableRdd = tableReader.makeRDDForTable(table) + return createPrunedRdd(databaseName, tableName, tableRdd) } - return createPrunedRdd(databaseName, tableName, SharkEnv.tachyonUtil.createRDD(tableName)) } else { // Table is a Hive table on HDFS (or other Hadoop storage). return makeRDDFromHadoop() @@ -224,7 +204,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { logDebug("Making %d Hive partitions".format(parts.size)) // The returned RDD contains arrays of size two with the elements as // (deserialized row, column partition value). - return hadoopReader.makeRDDForTablePartitions(parts) + return hadoopReader.makeRDDForPartitionedTable(parts) } else { // The returned RDD contains deserialized row Objects. return hadoopReader.makeRDDForTable(table) From 2cf0c4aa61c778ab6062d68d521b9547d9f07a65 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 31 Oct 2013 01:04:33 -0700 Subject: [PATCH 190/331] Some style cleanup in TableReader and HadoopTableReader. --- .../shark/execution/HadoopTableReader.scala | 8 ++--- .../scala/shark/execution/TableReader.scala | 30 +++++++++++-------- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index 0547aa46..87d7a6ee 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -23,12 +23,12 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} import org.apache.hadoop.io.Writable -import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.SerializableWritable import shark.{SharkEnv, Utils} @@ -93,7 +93,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf deserializedHadoopRDD } - override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = makeRDDForPartitionedTable(partitions, filterOpt = None) /** @@ -102,7 +102,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf * 'PARTITION BY'. */ def makeRDDForPartitionedTable( - partitions: Seq[Partition], + partitions: Seq[HivePartition], filterOpt: Option[PathFilter]): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 3791a2cb..22836562 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -22,17 +22,17 @@ import org.apache.hadoop.fs.PathFilter import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.SerializableWritable +import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.api.QueryExecutionException import shark.execution.optimization.ColumnPruner import shark.execution.serialization.JavaSerializer -import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.memstore2.{MemoryMetadataManager, TablePartition, TablePartitionStats} import shark.tachyon.TachyonException @@ -46,7 +46,7 @@ trait TableReader extends LogHelper{ def makeRDDForTable(hiveTable: HiveTable): RDD[_] - def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] + def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] } @@ -66,12 +66,14 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { } logInfo("Loading table " + tableKey + " from Tachyon.") - var indexToStats: collection.Map[Int, TablePartitionStats] = - SharkEnv.memoryMetadataManager.getStats(_databaseName, _tableName).getOrElse(null) - - if (indexToStats == null) { + // True if stats for the target table is missing from the Shark metastore, and should be fetched + // and deserialized from Tachyon's metastore. This can happen if that table was created in a + // previous Shark session, since Shark's metastore is not persistent. + var shouldFetchStatsFromTachyon = SharkEnv.memoryMetadataManager.getStats( + _databaseName, _tableName).isEmpty + if (shouldFetchStatsFromTachyon) { val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) - indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( + val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( statsByteBuffer.array()) logInfo("Loading table " + tableKey + " stats from Tachyon.") SharkEnv.memoryMetadataManager.putStats(_databaseName, _tableName, indexToStats) @@ -79,7 +81,7 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { SharkEnv.tachyonUtil.createRDD(tableKey) } - override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = { + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") } } @@ -96,7 +98,9 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { logInfo("Loading table %s.%s from Spark block manager".format(_databaseName, _tableName)) val tableOpt = SharkEnv.memoryMetadataManager.getMemoryTable(_databaseName, _tableName) - if (tableOpt.isEmpty) throwMissingTableException() + if (tableOpt.isEmpty) { + throwMissingTableException() + } val table = tableOpt.get table.tableRDD @@ -109,7 +113,7 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. */ - override def makeRDDForPartitionedTable(partitions: Seq[Partition]): RDD[_] = { + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) // Get partition field info @@ -129,7 +133,9 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) val hivePartitionedTableOpt = SharkEnv.memoryMetadataManager.getPartitionedTable( _databaseName, _tableName) - if (hivePartitionedTableOpt.isEmpty) throwMissingTableException() + if (hivePartitionedTableOpt.isEmpty) { + throwMissingTableException() + } val hivePartitionedTable = hivePartitionedTableOpt.get val hivePartitionRDDOpt = hivePartitionedTable.getPartition(partitionKeyStr) From cdaf5781fdfc7faebf7146ff76790dd92a39fb2c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 31 Oct 2013 23:06:50 -0700 Subject: [PATCH 191/331] Bug in MemoryStoreSinkOperator - UnionRDDs aren't actually flattened... --- .../execution/MemoryStoreSinkOperator.scala | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 7eec7f9c..f38849d9 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -175,7 +175,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { // of the current data and the SELECT output. hasPreviousRDDForUnion = true RDDUtils.unionAndFlatten(queryOutputRDD, previousRDD) - queryOutputRDD.union(previousRDD.asInstanceOf[RDD[TablePartition]]) } // This is an INSERT into a new Hive-partition. case None => queryOutputRDD @@ -212,23 +211,22 @@ class MemoryStoreSinkOperator extends TerminalOperator { // TODO(harvey): Get this to work for Hive-partitioned tables. It should be a simple // 'tableName' + 'hivePartitionKey' concatentation. Though whether stats should belong in // memstore2.Table should be considered... - val columnStats = - if (useUnionRDD && hasPreviousRDDForUnion) { - // Combine stats for the two RDDs that were combined into UnionRDD. - val numPartitions = statsAcc.value.toMap.size - val currentStats = statsAcc.value - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { - case Some(otherIndexToStats) => { - for ((otherIndex, tableStats) <- otherIndexToStats) { - currentStats.append((otherIndex + numPartitions, tableStats)) - } + val columnStats = if (useUnionRDD && hasPreviousRDDForUnion) { + // Combine stats for the two RDDs that were combined into UnionRDD. + val numPartitions = statsAcc.value.toMap.size + val currentStats = statsAcc.value + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { + case Some(otherIndexToStats) => { + for ((otherIndex, tableStats) <- otherIndexToStats) { + currentStats.append((otherIndex + numPartitions, tableStats)) } - case _ => Unit } - currentStats.toMap - } else { - statsAcc.value.toMap + case _ => Unit } + currentStats.toMap + } else { + statsAcc.value.toMap + } // Get the column statistics back to the cache manager. SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, columnStats) From 8ea073c5eda9ab3c79058640f596c4184b51f3d0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 31 Oct 2013 23:32:36 -0700 Subject: [PATCH 192/331] Unit test for UnionRDD "flattening". --- src/test/scala/shark/SQLSuite.scala | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 3f56d5e8..0ffaefab 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.storage.StorageLevel +import org.apache.spark.rdd.UnionRDD import shark.api.QueryExecutionException import shark.memstore2.PartitionedMemoryTable @@ -667,6 +668,29 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) } + /////////////////////////////////////////////////////////////////////////////////////// + // Prevent nested UnionRDDs - those should be "flattened" in MemoryStoreSinkOperator. + /////////////////////////////////////////////////////////////////////////////////////// + + test("flatten UnionRDDs") { + def isFlattenedUnionRDD(unionRDD: UnionRDD[_]) = { + unionRDD.rdds.filter(_.isInstanceOf[UnionRDD[_]]).isEmpty + } + + sc.sql("insert into table test_cached select * from test") + val tableName = "test_cached" + var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(DEFAULT_DB_NAME, tableName).get + var unionRDD = memoryTable.tableRDD.asInstanceOf[UnionRDD[_]] + val numParentRDDs = unionRDD.rdds.size + assert(isFlattenedUnionRDD(unionRDD)) + + // Insert another set of query results. The flattening should kick in here. + sc.sql("insert into table test_cached select * from test") + unionRDD = memoryTable.tableRDD.asInstanceOf[UnionRDD[_]] + assert(isFlattenedUnionRDD(unionRDD)) + assert(unionRDD.rdds.size == numParentRDDs + 1) + } + ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// From b174c1e9558cd16f43f8712700ef0d83e4cf6834 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 31 Oct 2013 23:35:47 -0700 Subject: [PATCH 193/331] Import ordering... --- src/test/scala/shark/SQLSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 0ffaefab..3a8e7a32 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -20,8 +20,8 @@ package shark import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.UnionRDD +import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.PartitionedMemoryTable From 038ef8bc53425c4c4c7fea49fd83bf347746170c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 31 Oct 2013 23:39:43 -0700 Subject: [PATCH 194/331] filter() => find() in SQLSuite when searching for nested UnionRDDs. --- src/test/scala/shark/SQLSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 3a8e7a32..22ddf701 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -674,7 +674,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("flatten UnionRDDs") { def isFlattenedUnionRDD(unionRDD: UnionRDD[_]) = { - unionRDD.rdds.filter(_.isInstanceOf[UnionRDD[_]]).isEmpty + unionRDD.rdds.find(_.isInstanceOf[UnionRDD[_]]).isEmpty } sc.sql("insert into table test_cached select * from test") From f98efeeb7e303de93904601eb2feab31ab32d67c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 1 Nov 2013 00:11:14 -0700 Subject: [PATCH 195/331] Unit tests for flattening UnionRDDs for partitioned tables. --- src/test/scala/shark/SQLSuite.scala | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 22ddf701..a318f959 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -691,6 +691,28 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(unionRDD.rdds.size == numParentRDDs + 1) } + test("flatten UnionRDDs for partitioned tables") { + sc.sql("drop table if exists part_table_cached") + sc.sql("""create table part_table_cached(key int, value string) + partitioned by (keypart int)""") + sc.sql("alter table part_table_cached add partition(keypart = 1)") + sc.sql("insert into table part_table_cached partition(keypart = 1) select * from flat_cached") + val tableName = "part_table_cached" + val partitionKey = "keypart=1" + var partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get + var unionRDD = partitionedTable.keyToPartitions.get(partitionKey).get.asInstanceOf[UnionRDD[_]] + val numParentRDDs = unionRDD.rdds.size + assert(isFlattenedUnionRDD(unionRDD)) + + // Insert another set of query results into the same partition. + // The flattening should kick in here. + sc.runSql("insert into table part_table_cached partition(keypart = 1) select * from flat_cached") + unionRDD = partitionedTable.getPartition(partitionKey).get.asInstanceOf[UnionRDD[_]] + assert(isFlattenedUnionRDD(unionRDD)) + assert(unionRDD.rdds.size == numParentRDDs + 1) + } + ////////////////////////////////////////////////////////////////////////////// // Tableau bug ////////////////////////////////////////////////////////////////////////////// From ed046e5523373524e5f6394600d397ab3f59b476 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 1 Nov 2013 00:11:30 -0700 Subject: [PATCH 196/331] Don't modify 'test_cached' table in SQLSuite... --- src/test/scala/shark/SQLSuite.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index a318f959..d3832478 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -138,6 +138,10 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { return partitionedTable } + def isFlattenedUnionRDD(unionRDD: UnionRDD[_]) = { + unionRDD.rdds.find(_.isInstanceOf[UnionRDD[_]]).isEmpty + } + ////////////////////////////////////////////////////////////////////////////// // basic SQL ////////////////////////////////////////////////////////////////////////////// @@ -673,19 +677,16 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { /////////////////////////////////////////////////////////////////////////////////////// test("flatten UnionRDDs") { - def isFlattenedUnionRDD(unionRDD: UnionRDD[_]) = { - unionRDD.rdds.find(_.isInstanceOf[UnionRDD[_]]).isEmpty - } - - sc.sql("insert into table test_cached select * from test") - val tableName = "test_cached" + sc.sql("create table flat_cached as select * from test_cached") + sc.sql("insert into table flat_cached select * from test") + val tableName = "flat_cached" var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(DEFAULT_DB_NAME, tableName).get var unionRDD = memoryTable.tableRDD.asInstanceOf[UnionRDD[_]] val numParentRDDs = unionRDD.rdds.size assert(isFlattenedUnionRDD(unionRDD)) // Insert another set of query results. The flattening should kick in here. - sc.sql("insert into table test_cached select * from test") + sc.sql("insert into table flat_cached select * from test") unionRDD = memoryTable.tableRDD.asInstanceOf[UnionRDD[_]] assert(isFlattenedUnionRDD(unionRDD)) assert(unionRDD.rdds.size == numParentRDDs + 1) From a9caa5acb9bcbc4fca53f817d3b75aa50a856ce7 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 29 Oct 2013 21:01:22 -0700 Subject: [PATCH 197/331] Add a SparkLoadWork, SparkLoadTask for supporting LOAD DATA INTO for cached tables. --- src/main/scala/shark/SharkDriver.scala | 5 +- .../execution/MemoryStoreSinkOperator.scala | 8 +- .../scala/shark/execution/SparkLoadTask.scala | 181 ++++++++++++++++++ .../scala/shark/execution/TableReader.scala | 2 +- .../parse/SharkDDLSemanticAnalyzer.scala | 3 +- .../parse/SharkLoadSemanticAnalyzer.scala | 104 ++++++++++ .../parse/SharkSemanticAnalyzerFactory.scala | 4 +- 7 files changed, 298 insertions(+), 9 deletions(-) create mode 100644 src/main/scala/shark/execution/SparkLoadTask.scala create mode 100644 src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 1869c946..1718a51c 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -35,8 +35,8 @@ import org.apache.hadoop.util.StringUtils import shark.api.TableRDD import shark.api.QueryExecutionException -import shark.execution.{SharkDDLTask, SharkDDLWork, SharkExplainTask, SharkExplainWork, SparkTask, - SparkWork} +import shark.execution.{SharkDDLTask, SharkDDLWork, SharkExplainTask, SharkExplainWork, + SparkLoadWork, SparkLoadTask, SparkTask, SparkWork} import shark.memstore2.ColumnarSerDe import shark.parse.{QueryContext, SharkSemanticAnalyzerFactory} @@ -64,6 +64,7 @@ private[shark] object SharkDriver extends LogHelper { // Task factory. Add Shark specific tasks. TaskFactory.taskvec.addAll(Seq( new TaskFactory.taskTuple(classOf[SharkDDLWork], classOf[SharkDDLTask]), + new TaskFactory.taskTuple(classOf[SparkLoadWork], classOf[SparkLoadTask]), new TaskFactory.taskTuple(classOf[SparkWork], classOf[SparkTask]), new TaskFactory.taskTuple(classOf[SharkExplainWork], classOf[SharkExplainTask]))) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index f38849d9..5d4d63ab 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -113,13 +113,13 @@ class MemoryStoreSinkOperator extends TerminalOperator { builder = serde.serialize(row.asInstanceOf[AnyRef], op.objectInspector) } - if (builder != null) { - statsAcc += Tuple2(part, builder.asInstanceOf[TablePartitionBuilder].stats) - Iterator(builder.asInstanceOf[TablePartitionBuilder].build) - } else { + if (builder == null) { // Empty partition. statsAcc += Tuple2(part, new TablePartitionStats(Array(), 0)) Iterator(new TablePartition(0, Array())) + } else { + statsAcc += Tuple2(part, builder.asInstanceOf[TablePartitionBuilder].stats) + Iterator(builder.asInstanceOf[TablePartitionBuilder].build) } } diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala new file mode 100644 index 00000000..15d57b3b --- /dev/null +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -0,0 +1,181 @@ +/** + * 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 shark.execution + +import java.util.{Properties, Map => JavaMap} + +import scala.collection.JavaConversions._ + +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.{Context, DriverContext} +import org.apache.hadoop.hive.ql.exec.{Task => HiveTask, Utilities} +import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} +import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.ql.plan.api.StageType +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat} + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.SerializableWritable + +import shark.{LogHelper, SharkEnv} +import shark.execution.serialization.KryoSerializer +import shark.memstore2._ + + +private[shark] +class SparkLoadWork( + val hiveTable: HiveTable, + val partSpecsOpt: Option[JavaMap[String, String]], + val isOverwrite: Boolean, + val pathFilter: Option[PathFilter]) + extends java.io.Serializable + +private[shark] +class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHelper { + + override def execute(driveContext: DriverContext): Int = { + logDebug("Executing " + this.getClass.getName) + + val hiveTable = work.hiveTable + val tableDesc = Utilities.getTableDesc(hiveTable) + val tableNameSplit = tableDesc.getTableName().split('.') + val databaseName = tableNameSplit(0) + val tableName = tableNameSplit(1) + val oi = hiveTable.getDeserializer().getObjectInspector().asInstanceOf[StructObjectInspector] + + val hadoopReader = new HadoopTableReader(tableDesc, conf) + val inputRDD = if (hiveTable.isPartitioned) { + val partition = db.getPartition(hiveTable, work.partSpecsOpt.get, false /* forceCreate */) + hadoopReader.makeRDDForPartitionedTable(Seq(partition), work.pathFilter) + } else { + hadoopReader.makeRDDForTable(hiveTable, work.pathFilter) + } + + val (tablePartitionRDD, tableStats) = transformInputRdd( + inputRDD, + tableDesc.getProperties, + hadoopReader.broadcastedHiveConf, + hiveTable.getDeserializer.getObjectInspector().asInstanceOf[StructObjectInspector]) + + // TODO(harvey): This part overlaps with, though is slightly cleaner than, execution code in + // MemoryStoreSinkOperator. + // LOAD DATA INPATH behaves like an INSERT, so use a UnionRDD if no OVERWRITE is specified. + SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { + case Some(table) => { + var unionWithPreviousStats = false + table match { + case memoryTable: MemoryTable => { + memoryTable.tableRDD = if (work.isOverwrite) { + tablePartitionRDD + } else { + unionWithPreviousStats = true + RDDUtils.unionAndFlatten(memoryTable.tableRDD, tablePartitionRDD) + } + } + case partitionedTable: PartitionedMemoryTable => { + val partCols = hiveTable.getPartCols.map(_.getName) + val partSpecs = work.partSpecsOpt.get + val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpecs) + partitionedTable.getPartition(partitionKey) match { + case Some(previousRDD) => { + partitionedTable.updatePartition( + partitionKey, RDDUtils.unionAndFlatten(previousRDD, tablePartitionRDD)) + unionWithPreviousStats = true + } + case None => { + // Either `isOverwrite` is true, or the partition being updated does not currently + // exist. + partitionedTable.putPartition(partitionKey, tablePartitionRDD) + } + } + } + } + val newStats = if (unionWithPreviousStats) { + // Union the stats maps. + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { + case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) + case _ => tableStats + } + } else { + tableStats + } + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, newStats.toMap) + } + case None => { + throw new Exception("Couldn't find the target table for a LOAD.") + } + } + + // Success! + 0 + } + + def transformInputRdd( + inputRdd: RDD[_], + tableProps: Properties, + broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], + oi: StructObjectInspector) = { + val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) + val serializedOI = KryoSerializer.serialize(oi) + val transformedRdd = inputRdd.mapPartitionsWithIndex { case (partIndex, partIter) => + val serde = new ColumnarSerDe + serde.initialize(broadcastedHiveConf.value.value, tableProps) + val oi = KryoSerializer.deserialize[ObjectInspector](serializedOI) + var builder: Writable = null + partIter.foreach { row => + builder = serde.serialize(row.asInstanceOf[AnyRef], oi) + } + if (builder == null) { + // Empty partition. + statsAcc += Tuple2(partIndex, new TablePartitionStats(Array(), 0)) + Iterator(new TablePartition(0, Array())) + } else { + statsAcc += Tuple2(partIndex, builder.asInstanceOf[TablePartitionBuilder].stats) + Iterator(builder.asInstanceOf[TablePartitionBuilder].build) + } + } + transformedRdd.context.runJob( + transformedRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + (transformedRdd, statsAcc.value) + } + + def unionStatsMaps( + targetStatsMap: ArrayBuffer[(Int, TablePartitionStats)], + otherStatsMap: Iterable[(Int, TablePartitionStats)] + ): ArrayBuffer[(Int, TablePartitionStats)] = { + val targetStatsMapSize = targetStatsMap.size + for ((otherIndex, tableStats) <- otherStatsMap) { + targetStatsMap.append((otherIndex + targetStatsMapSize, tableStats)) + } + targetStatsMap + } + + override def getType = StageType.MAPRED + + override def getName = "MAPRED-LOAD-SPARK" + + override def localizeMRTmpFilesImpl(ctx: Context) = Unit + +} diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 22836562..0297ab81 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -25,8 +25,8 @@ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} -import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} import org.apache.spark.SerializableWritable import shark.{LogHelper, SharkConfVars, SharkEnv} diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 97a55c8c..9f1add57 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -4,7 +4,8 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.TaskFactory -import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, HiveParser} +import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, + HiveParser} import org.apache.hadoop.hive.ql.plan.DDLWork import org.apache.spark.rdd.{UnionRDD, RDD} diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala new file mode 100644 index 00000000..c5d8d16d --- /dev/null +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.parse + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.exec.{CopyTask, MoveTask, TaskFactory} +import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} +import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, LoadSemanticAnalyzer} +import org.apache.hadoop.hive.ql.plan._ + +import shark.execution.SparkLoadWork +import shark.{LogHelper, SharkEnv} + +class SharkLoadSemanticAnalyzer(hiveConf: HiveConf) extends LoadSemanticAnalyzer(hiveConf) { + + override def analyzeInternal(ast: ASTNode): Unit = { + // Delegate to the LoadSemanticAnalyzer parent for error checking the source path formatting. + super.analyzeInternal(ast) + + // Children of the AST root created for a LOAD DATA [LOCAL] INPATH ... statement are, in order: + // 1. node containing the path specified by INPATH. + // 2. internal TOK_TABNAME node that contains the table's name. + // 3. (optional) node representing the LOCAL modifier. + val tableASTNode = ast.getChild(1).asInstanceOf[ASTNode] + val tableName = getTableName(tableASTNode) + val databaseName = db.getCurrentDatabase() + + if (SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { + // Find the arguments needed to instantiate a SparkLoadWork. + val tableSpec = new BaseSemanticAnalyzer.tableSpec(db, conf, tableASTNode) + val hiveTable = tableSpec.tableHandle + val partSpecOpt = Option(tableSpec.getPartSpec()) + var partition: Partition = null + val dataPath = if (partSpecOpt.isEmpty) { + // Non-partitioned table. + hiveTable.getPath + } else { + // Partitioned table. + partition = db.getPartition(hiveTable, partSpecOpt.get, false /* forceCreate */) + partition.getPartitionPath + } + val moveTask = getMoveTask() + val isOverwrite = moveTask.getWork.getLoadTableWork.getReplace() + + // Capture a snapshot of the data directory being read. When executed, SparkLoadTask will + // determine the input paths to read using a filter that only accepts files not included in + // snapshot set (i.e., the accepted file is a new one created by the Hive load process). + val fs = dataPath.getFileSystem(hiveConf) + val currentFiles = fs.listStatus(dataPath).map(_.getPath).toSet + val fileFilter = new PathFilter() { + override def accept(path: Path) = { + (!path.getName().startsWith(".") && !currentFiles.contains(path)) + } + } + + // Create a SparkLoadTask that will use a HadoopRDD to read from the source directory. Set it + // to be a dependent task of the LoadTask so that the SparkLoadTask is executed only if the + // Hive task executes successfully. + val sparkLoadWork = new SparkLoadWork(hiveTable, partSpecOpt, isOverwrite, Some(fileFilter)) + moveTask.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) + } + } + + private def getMoveTask(): MoveTask = { + assert(rootTasks.size == 1) + + // If the execution is local, a CopyTask will be the root task, with a MoveTask child. + // Otherwise, a MoveTask will be the root. + var rootTask = rootTasks.head + val moveTask = if (rootTask.isInstanceOf[CopyTask]) { + val firstChildTask = rootTask.getChildTasks.head + assert(firstChildTask.isInstanceOf[MoveTask]) + firstChildTask + } else { + rootTask + } + + // In Hive, LoadTableDesc is referred to as LoadTableWork... + moveTask.asInstanceOf[MoveTask] + } + + private def getTableName(node: ASTNode): String = { + BaseSemanticAnalyzer.getUnescapedName(node.getChild(0).asInstanceOf[ASTNode]) + } +} + diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala index 91215988..5bf17386 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala @@ -19,7 +19,7 @@ package shark.parse import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, DDLSemanticAnalyzer, - SemanticAnalyzerFactory, ExplainSemanticAnalyzer, SemanticAnalyzer} + ExplainSemanticAnalyzer, LoadSemanticAnalyzer, SemanticAnalyzerFactory, SemanticAnalyzer} import shark.SharkConfVars @@ -39,6 +39,8 @@ object SharkSemanticAnalyzerFactory { new SharkExplainSemanticAnalyzer(conf) } else if (baseSem.isInstanceOf[DDLSemanticAnalyzer]) { new SharkDDLSemanticAnalyzer(conf) + } else if (baseSem.isInstanceOf[LoadSemanticAnalyzer]) { + new SharkLoadSemanticAnalyzer(conf) } else { baseSem } From 5e5f9d66d6948df4ce2e950ee670ce6ea25e0d46 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 1 Nov 2013 23:33:29 -0700 Subject: [PATCH 198/331] Use the proper local variable for fair scheduler pool in Spark. --- src/main/scala/shark/execution/SparkTask.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index 6aa9ffc7..b9e828ec 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -92,7 +92,7 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { SharkEnv.sc.setJobDescription(work.pctx.getContext.getCmd) // Set the fair scheduler's pool. - SharkEnv.sc.setLocalProperty("spark.scheduler.cluster.fair.pool", + SharkEnv.sc.setLocalProperty("spark.scheduler.pool", conf.get("mapred.fairscheduler.pool")) val sinkRdd = terminalOp.execute().asInstanceOf[RDD[Any]] From 924b6422b9691a9ddbc75284fb80268e27a903bc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 2 Nov 2013 00:52:53 -0700 Subject: [PATCH 199/331] Various cleanup. --- src/main/scala/shark/SharkEnv.scala | 26 ++++++++-------- src/main/scala/shark/api/Row.scala | 1 - .../shark/execution/CommonJoinOperator.scala | 10 +----- .../shark/execution/ExtractOperator.scala | 3 -- .../shark/execution/FileSinkOperator.scala | 6 ++-- .../shark/execution/FilterOperator.scala | 1 - .../shark/execution/ForwardOperator.scala | 2 -- .../shark/execution/GroupByOperator.scala | 1 - .../shark/execution/HadoopTableReader.scala | 8 ++--- .../scala/shark/execution/JoinOperator.scala | 5 ++- .../LateralViewForwardOperator.scala | 1 - .../execution/LateralViewJoinOperator.scala | 2 -- .../scala/shark/execution/LimitOperator.scala | 4 --- .../shark/execution/MapJoinOperator.scala | 4 --- .../shark/execution/MapSplitPruning.scala | 4 --- .../execution/MemoryStoreSinkOperator.scala | 8 ++--- src/main/scala/shark/execution/Operator.scala | 4 +-- .../shark/execution/OperatorFactory.scala | 15 +++++---- src/main/scala/shark/execution/RDDUtils.scala | 4 +-- .../shark/execution/ReduceSinkOperator.scala | 8 ++--- .../shark/execution/ReduceSinkTableDesc.scala | 9 +++--- .../shark/execution/ScriptOperator.scala | 10 +++--- .../shark/execution/SelectOperator.scala | 1 - .../scala/shark/execution/SharkDDLTask.scala | 4 +-- .../shark/execution/SharkExplainTask.scala | 10 +++--- .../scala/shark/execution/TableReader.scala | 16 +++------- .../shark/execution/TableScanOperator.scala | 26 ++++++++-------- .../scala/shark/execution/UDTFOperator.scala | 6 ++-- .../scala/shark/execution/UnionOperator.scala | 3 -- .../execution/optimization/ColumnPruner.scala | 17 +++++----- .../HiveStructDeserializer.scala | 2 -- .../serialization/HiveStructSerializer.scala | 2 -- .../serialization/XmlSerializer.scala | 6 ++-- .../scala/shark/memstore2/CachePolicy.scala | 2 -- .../ColumnarStructObjectInspector.scala | 2 -- .../memstore2/MemoryMetadataManager.scala | 25 +++++++-------- .../scala/shark/memstore2/MemoryTable.scala | 2 -- .../memstore2/PartitionedMemoryTable.scala | 31 ++++++++++--------- .../memstore2/TablePartitionIterator.scala | 4 +-- .../parse/SharkDDLSemanticAnalyzer.scala | 3 -- .../parse/SharkExplainSemanticAnalyzer.scala | 2 -- .../shark/parse/SharkSemanticAnalyzer.scala | 10 +++--- .../parse/SharkSemanticAnalyzerFactory.scala | 20 ++++++------ src/main/scala/shark/util/BloomFilter.scala | 14 ++++----- src/main/scala/shark/util/HiveUtils.scala | 6 ++-- .../shark/util/MurmurHash3_x86_128.scala | 3 +- 46 files changed, 142 insertions(+), 211 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index c8b28706..549817f6 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -19,10 +19,9 @@ package shark import scala.collection.mutable.{HashMap, HashSet} -import org.apache.spark.{SparkContext, SparkEnv} +import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.StatsReportListener -import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} import shark.api.JavaSharkContext import shark.memstore2.MemoryMetadataManager @@ -36,11 +35,11 @@ object SharkEnv extends LogHelper { def init(): SparkContext = { if (sc == null) { sc = new SparkContext( - if (System.getenv("MASTER") == null) "local" else System.getenv("MASTER"), - "Shark::" + java.net.InetAddress.getLocalHost.getHostName, - System.getenv("SPARK_HOME"), - Nil, - executorEnvVars) + if (System.getenv("MASTER") == null) "local" else System.getenv("MASTER"), + "Shark::" + java.net.InetAddress.getLocalHost.getHostName, + System.getenv("SPARK_HOME"), + Nil, + executorEnvVars) sc.addSparkListener(new StatsReportListener()) } sc @@ -53,11 +52,11 @@ object SharkEnv extends LogHelper { } sc = new SharkContext( - if (master == null) "local" else master, - jobName, - System.getenv("SPARK_HOME"), - Nil, - executorEnvVars) + if (master == null) "local" else master, + jobName, + System.getenv("SPARK_HOME"), + Nil, + executorEnvVars) sc.addSparkListener(new StatsReportListener()) sc.asInstanceOf[SharkContext] } @@ -116,7 +115,8 @@ object SharkEnv extends LogHelper { * delegating to MemoryMetadataManager#removeTable() for removing the table's entry from the * Shark metastore. * - * @param tableName The table that should be dropped from the Shark metastore and from memory storage. + * @param tableName The table that should be dropped from the Shark metastore and from memory + * storage. */ def dropTable(databaseName: String, tableName: String): Option[RDD[_]] = { val tableKey = makeTachyonTableKey(databaseName, tableName) diff --git a/src/main/scala/shark/api/Row.scala b/src/main/scala/shark/api/Row.scala index b6f2224a..1c1c2c91 100644 --- a/src/main/scala/shark/api/Row.scala +++ b/src/main/scala/shark/api/Row.scala @@ -18,7 +18,6 @@ package shark.api import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ /** diff --git a/src/main/scala/shark/execution/CommonJoinOperator.scala b/src/main/scala/shark/execution/CommonJoinOperator.scala index 8cb055fc..ff60ac3a 100755 --- a/src/main/scala/shark/execution/CommonJoinOperator.scala +++ b/src/main/scala/shark/execution/CommonJoinOperator.scala @@ -19,22 +19,14 @@ package shark.execution import java.util.{HashMap => JavaHashMap, List => JavaList, ArrayList =>JavaArrayList} -import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ import scala.reflect.BeanProperty -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator -import org.apache.hadoop.hive.ql.exec.{CommonJoinOperator => HiveCommonJoinOperator} import org.apache.hadoop.hive.ql.exec.{JoinUtil => HiveJoinUtil} -import org.apache.hadoop.hive.ql.plan.{ExprNodeDesc, JoinCondDesc, JoinDesc, TableDesc} -import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.ql.plan.{JoinCondDesc, JoinDesc} import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, PrimitiveObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory -import org.apache.spark.rdd.{RDD, UnionRDD} -import org.apache.spark.SparkContext.rddToPairRDDFunctions - import shark.SharkConfVars diff --git a/src/main/scala/shark/execution/ExtractOperator.scala b/src/main/scala/shark/execution/ExtractOperator.scala index fe175d1b..767f6573 100755 --- a/src/main/scala/shark/execution/ExtractOperator.scala +++ b/src/main/scala/shark/execution/ExtractOperator.scala @@ -22,15 +22,12 @@ import scala.reflect.BeanProperty import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} -import org.apache.hadoop.hive.ql.exec.{ExtractOperator => HiveExtractOperator} import org.apache.hadoop.hive.ql.plan.{ExtractDesc, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.io.BytesWritable import org.apache.spark.rdd.RDD -import shark.SharkConfVars - class ExtractOperator extends UnaryOperator[ExtractDesc] with ReduceSinkTableDesc { diff --git a/src/main/scala/shark/execution/FileSinkOperator.scala b/src/main/scala/shark/execution/FileSinkOperator.scala index c0f6d868..d26142fb 100644 --- a/src/main/scala/shark/execution/FileSinkOperator.scala +++ b/src/main/scala/shark/execution/FileSinkOperator.scala @@ -28,9 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} import org.apache.hadoop.hive.ql.exec.JobCloseFeedBack import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.hadoop.mapred.{JobID, TaskID} -import org.apache.hadoop.mapred.TaskAttemptID -import org.apache.hadoop.mapred.SparkHadoopWriter +import org.apache.hadoop.mapred.{JobID, TaskAttemptID, TaskID} import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -61,7 +59,7 @@ class FileSinkOperator extends TerminalOperator with Serializable { } val jobID = context.stageId - val splitID = context.splitId + val splitID = context.partitionId val jID = createJobID(now, jobID) val taID = new TaskAttemptID(new TaskID(jID, true, splitID), 0) conf.set("mapred.job.id", jID.toString) diff --git a/src/main/scala/shark/execution/FilterOperator.scala b/src/main/scala/shark/execution/FilterOperator.scala index 20228986..7dda42d0 100755 --- a/src/main/scala/shark/execution/FilterOperator.scala +++ b/src/main/scala/shark/execution/FilterOperator.scala @@ -21,7 +21,6 @@ import scala.collection.Iterator import scala.reflect.BeanProperty import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} -import org.apache.hadoop.hive.ql.exec.{FilterOperator => HiveFilterOperator} import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.plan.FilterDesc import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector diff --git a/src/main/scala/shark/execution/ForwardOperator.scala b/src/main/scala/shark/execution/ForwardOperator.scala index e41d8f61..93e1ab4d 100755 --- a/src/main/scala/shark/execution/ForwardOperator.scala +++ b/src/main/scala/shark/execution/ForwardOperator.scala @@ -17,8 +17,6 @@ package shark.execution -import org.apache.hadoop.hive.ql.exec.{ForwardOperator => HiveForwardOperator} - import org.apache.spark.rdd.RDD import org.apache.hadoop.hive.ql.plan.ForwardDesc diff --git a/src/main/scala/shark/execution/GroupByOperator.scala b/src/main/scala/shark/execution/GroupByOperator.scala index 8e9fe517..db65b990 100755 --- a/src/main/scala/shark/execution/GroupByOperator.scala +++ b/src/main/scala/shark/execution/GroupByOperator.scala @@ -19,7 +19,6 @@ package shark.execution import org.apache.hadoop.hive.ql.exec.{GroupByOperator => HiveGroupByOperator} import org.apache.hadoop.hive.ql.exec.{ReduceSinkOperator => HiveReduceSinkOperator} -import org.apache.hadoop.hive.ql.plan.GroupByDesc /** diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index 87d7a6ee..9018700b 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -18,14 +18,14 @@ package shark.execution import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} -import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} +import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} -import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} @@ -212,7 +212,7 @@ object HadoopTableReader { // Set s3/s3n credentials. Setting them in localJobConf ensures the settings propagate // from Spark's master all the way to Spark's slaves. var s3varsSet = false - val s3vars = Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", + Seq("fs.s3n.awsAccessKeyId", "fs.s3n.awsSecretAccessKey", "fs.s3.awsAccessKeyId", "fs.s3.awsSecretAccessKey").foreach { variableName => if (conf.get(variableName) != null) { s3varsSet = true diff --git a/src/main/scala/shark/execution/JoinOperator.scala b/src/main/scala/shark/execution/JoinOperator.scala index acea4497..21592b67 100755 --- a/src/main/scala/shark/execution/JoinOperator.scala +++ b/src/main/scala/shark/execution/JoinOperator.scala @@ -24,9 +24,8 @@ import scala.collection.JavaConversions._ import scala.reflect.BeanProperty import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.exec.{JoinOperator => HiveJoinOperator} import org.apache.hadoop.hive.ql.plan.{JoinDesc, TableDesc} -import org.apache.hadoop.hive.serde2.{Deserializer, Serializer, SerDeUtils} +import org.apache.hadoop.hive.serde2.{Deserializer, SerDeUtils} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector import org.apache.hadoop.io.BytesWritable @@ -48,7 +47,7 @@ class JoinOperator extends CommonJoinOperator[JoinDesc] with ReduceSinkTableDesc override def initializeOnMaster() { super.initializeOnMaster() - var descs = keyValueDescs() + val descs = keyValueDescs() valueTableDescMap = new JHashMap[Int, TableDesc] valueTableDescMap ++= descs.map { case(tag, kvdescs) => (tag, kvdescs._2) } keyTableDesc = descs.head._2._1 diff --git a/src/main/scala/shark/execution/LateralViewForwardOperator.scala b/src/main/scala/shark/execution/LateralViewForwardOperator.scala index ec497d23..458bd7c3 100755 --- a/src/main/scala/shark/execution/LateralViewForwardOperator.scala +++ b/src/main/scala/shark/execution/LateralViewForwardOperator.scala @@ -17,7 +17,6 @@ package shark.execution -import org.apache.hadoop.hive.ql.exec.{LateralViewForwardOperator => HiveLateralViewForwardOperator} import org.apache.hadoop.hive.ql.plan.LateralViewForwardDesc import org.apache.spark.rdd.RDD diff --git a/src/main/scala/shark/execution/LateralViewJoinOperator.scala b/src/main/scala/shark/execution/LateralViewJoinOperator.scala index 2f2b5e39..b6dedefa 100755 --- a/src/main/scala/shark/execution/LateralViewJoinOperator.scala +++ b/src/main/scala/shark/execution/LateralViewJoinOperator.scala @@ -18,7 +18,6 @@ package shark.execution import java.nio.ByteBuffer -import java.util.ArrayList import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ @@ -26,7 +25,6 @@ import scala.reflect.BeanProperty import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} -import org.apache.hadoop.hive.ql.exec.{LateralViewJoinOperator => HiveLateralViewJoinOperator} import org.apache.hadoop.hive.ql.plan.SelectDesc import org.apache.hadoop.hive.ql.plan.LateralViewJoinDesc import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory diff --git a/src/main/scala/shark/execution/LimitOperator.scala b/src/main/scala/shark/execution/LimitOperator.scala index 5bd063fd..c78c0ab4 100755 --- a/src/main/scala/shark/execution/LimitOperator.scala +++ b/src/main/scala/shark/execution/LimitOperator.scala @@ -17,10 +17,6 @@ package shark.execution -import scala.collection.Iterator -import scala.reflect.BeanProperty - -import org.apache.hadoop.hive.ql.exec.{LimitOperator => HiveLimitOperator} import org.apache.hadoop.hive.ql.plan.LimitDesc import org.apache.spark.rdd.{EmptyRDD, RDD} diff --git a/src/main/scala/shark/execution/MapJoinOperator.scala b/src/main/scala/shark/execution/MapJoinOperator.scala index 0c5bc515..7ea9bea6 100755 --- a/src/main/scala/shark/execution/MapJoinOperator.scala +++ b/src/main/scala/shark/execution/MapJoinOperator.scala @@ -19,14 +19,10 @@ package shark.execution import java.util.{ArrayList, HashMap => JHashMap, List => JList} -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.BeanProperty -import org.apache.hadoop.io.BytesWritable - import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, JoinUtil => HiveJoinUtil} -import org.apache.hadoop.hive.ql.exec.{MapJoinOperator => HiveMapJoinOperator} import org.apache.hadoop.hive.ql.plan.MapJoinDesc import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index f89b8676..19cea0fd 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -17,8 +17,6 @@ package org.apache.hadoop.hive.ql.exec -import java.sql.Timestamp - import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual @@ -29,8 +27,6 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import org.apache.hadoop.io.Text import shark.memstore2.ColumnarStructObjectInspector.IDStructField import shark.memstore2.TablePartitionStats diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index f38849d9..f5e28129 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -115,7 +115,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { if (builder != null) { statsAcc += Tuple2(part, builder.asInstanceOf[TablePartitionBuilder].stats) - Iterator(builder.asInstanceOf[TablePartitionBuilder].build) + Iterator(builder.asInstanceOf[TablePartitionBuilder].build()) } else { // Empty partition. statsAcc += Tuple2(part, new TablePartitionStats(Array(), 0)) @@ -158,10 +158,10 @@ class MemoryStoreSinkOperator extends TerminalOperator { outputRDD.persist(storageLevel) - var queryOutputRDD = outputRDD + val queryOutputRDD = outputRDD if (useUnionRDD) { // Handle an INSERT INTO command. - var previousRDDOpt: Option[RDD[TablePartition]] = + val previousRDDOpt: Option[RDD[TablePartition]] = if (isHivePartitioned) { val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( databaseName, tableName).get @@ -202,7 +202,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { } else { outputRDD.setName(tableName) // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). - var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) + val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( databaseName, tableName, cacheMode, storageLevel)) memoryTable.tableRDD = outputRDD diff --git a/src/main/scala/shark/execution/Operator.scala b/src/main/scala/shark/execution/Operator.scala index 31200dce..2cd37ce3 100755 --- a/src/main/scala/shark/execution/Operator.scala +++ b/src/main/scala/shark/execution/Operator.scala @@ -17,7 +17,7 @@ package shark.execution -import java.util.{List => JavaList, Arrays} +import java.util.{List => JavaList} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ @@ -219,7 +219,7 @@ abstract class Operator[+T <: HiveDesc] extends LogHelper with Serializable { val result = new Array[ObjectInspector](length) for (i <- 0 to length - 1) { - result(i) = evals(start + i).initialize(rowInspector); + result(i) = evals(start + i).initialize(rowInspector) } result diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 66b4d592..a631da23 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -19,8 +19,7 @@ package shark.execution import scala.collection.JavaConversions._ -import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator} -import org.apache.hadoop.hive.ql.exec.GroupByPreShuffleOperator +import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator, GroupByPreShuffleOperator} import org.apache.hadoop.hive.ql.exec.{Operator => HOperator} import org.apache.hadoop.hive.ql.metadata.HiveException @@ -55,7 +54,7 @@ object OperatorFactory extends LogHelper { cacheMode: CacheType, useUnionRDD: Boolean): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? - var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] + val hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] val sinkOp = _newOperatorInstance( classOf[MemoryStoreSinkOperator], hiveOp).asInstanceOf[MemoryStoreSinkOperator] sinkOp.localHiveOp = hiveOp @@ -71,7 +70,7 @@ object OperatorFactory extends LogHelper { def createSharkFileOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? - var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] + val hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] val sinkOp = _newOperatorInstance(classOf[FileSinkOperator], hiveOp).asInstanceOf[TerminalOperator] sinkOp.localHiveOp = hiveOp @@ -80,7 +79,7 @@ object OperatorFactory extends LogHelper { def createSharkRddOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? - var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] + val hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] val sinkOp = _newOperatorInstance(classOf[TableRddSinkOperator], hiveOp).asInstanceOf[TableRddSinkOperator] sinkOp.localHiveOp = hiveOp @@ -92,13 +91,13 @@ object OperatorFactory extends LogHelper { // This is kind of annoying, but it works with strong typing ... val sharkOp = hiveOp match { case hop: org.apache.hadoop.hive.ql.exec.TableScanOperator => - var op = _newOperatorInstance(classOf[TableScanOperator], hop) + val op = _newOperatorInstance(classOf[TableScanOperator], hop) op.asInstanceOf[TableScanOperator].hiveOp = hop op case hop: org.apache.hadoop.hive.ql.exec.SelectOperator => _newOperatorInstance(classOf[SelectOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.FileSinkOperator => - var op = _newOperatorInstance(classOf[TerminalOperator], hop) + val op = _newOperatorInstance(classOf[TerminalOperator], hop) op.asInstanceOf[TerminalOperator].localHiveOp = hop op case hop: org.apache.hadoop.hive.ql.exec.LimitOperator => @@ -116,7 +115,7 @@ object OperatorFactory extends LogHelper { case hop: org.apache.hadoop.hive.ql.exec.MapJoinOperator => _newOperatorInstance(classOf[MapJoinOperator], hop) case hop: org.apache.hadoop.hive.ql.exec.ScriptOperator => - var op = _newOperatorInstance(classOf[ScriptOperator], hop) + val op = _newOperatorInstance(classOf[ScriptOperator], hop) op.asInstanceOf[ScriptOperator].operatorId = hop.getOperatorId() op case hop: org.apache.hadoop.hive.ql.exec.LateralViewForwardOperator => diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 7d18dd53..6ccd26af 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -38,7 +38,7 @@ object RDDUtils { otherRdd: RDD[T]): RDD[T] = { val unionedRdd = otherRdd match { case unionRdd: UnionRDD[_] => { - new UnionRDD(rdd.context, (unionRdd.rdds :+ rdd)) + new UnionRDD(rdd.context, unionRdd.rdds :+ rdd) } case _ => rdd.union(otherRdd) } @@ -85,7 +85,7 @@ object RDDUtils { shuffled.mapPartitions(iter => { val buf = iter.toArray buf.sortWith((x, y) => x._1.compareTo(y._1) < 0).iterator - }, true) + }, preservesPartitioning = true) } /** diff --git a/src/main/scala/shark/execution/ReduceSinkOperator.scala b/src/main/scala/shark/execution/ReduceSinkOperator.scala index 096bc0ed..619544a3 100755 --- a/src/main/scala/shark/execution/ReduceSinkOperator.scala +++ b/src/main/scala/shark/execution/ReduceSinkOperator.scala @@ -23,14 +23,12 @@ import scala.collection.Iterator import scala.collection.JavaConversions._ import scala.reflect.BeanProperty -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.exec.{ReduceSinkOperator => HiveReduceSinkOperator} import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} -import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc import org.apache.hadoop.hive.serde2.SerDe -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory, - ObjectInspectorUtils} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils import org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector.StandardUnion import org.apache.hadoop.io.BytesWritable diff --git a/src/main/scala/shark/execution/ReduceSinkTableDesc.scala b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala index c7952263..5acefae7 100644 --- a/src/main/scala/shark/execution/ReduceSinkTableDesc.scala +++ b/src/main/scala/shark/execution/ReduceSinkTableDesc.scala @@ -17,8 +17,9 @@ package shark.execution -import shark.LogHelper import org.apache.hadoop.hive.ql.plan.TableDesc +import shark.LogHelper + trait ReduceSinkTableDesc extends LogHelper { self: Operator[_ <: HiveDesc] => @@ -26,9 +27,9 @@ trait ReduceSinkTableDesc extends LogHelper { // Seq(tag, (Key TableDesc, Value TableDesc)) def keyValueDescs(): Seq[(Int, (TableDesc, TableDesc))] = { // get the parent ReduceSinkOperator and sort it by tag - val reduceSinkOps = - for (op <- self.parentOperators.toSeq; - if (op.isInstanceOf[ReduceSinkOperator])) yield op.asInstanceOf[ReduceSinkOperator] + val reduceSinkOps = + for (op <- self.parentOperators.toSeq if op.isInstanceOf[ReduceSinkOperator]) + yield op.asInstanceOf[ReduceSinkOperator] reduceSinkOps.map(f => (f.getTag, f.getKeyValueTableDescs)) } diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index ddbbd234..3de2a00e 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -19,7 +19,7 @@ package shark.execution import java.io.{File, InputStream, IOException} import java.lang.Thread.UncaughtExceptionHandler -import java.util.{Arrays, Properties} +import java.util.Properties import scala.collection.JavaConversions._ import scala.io.Source @@ -29,13 +29,11 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{ScriptOperator => HiveScriptOperator} import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter, ScriptOperatorHelper} -import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.plan.ScriptDesc import org.apache.hadoop.hive.serde2.{Serializer, Deserializer} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.io.{BytesWritable, Writable} -import org.apache.spark.{OneToOneDependency, SparkEnv, SparkFiles} +import org.apache.spark.{SparkEnv, SparkFiles} import org.apache.spark.rdd.RDD import org.apache.spark.TaskContext @@ -186,7 +184,7 @@ class ScriptOperator extends UnaryOperator[ScriptDesc] { if (!(new File(prog)).isAbsolute()) { val finder = scriptOpHelper.newPathFinderInstance("PATH") finder.prependPathComponent(currentDir.toString()) - var f = finder.getAbsolutePath(prog) + val f = finder.getAbsolutePath(prog) if (f != null) { cmdArgs(0) = f.getAbsolutePath() } @@ -341,7 +339,7 @@ object ScriptOperator { if (recordLength >= 0) { bytesWritable.setSize(recordLength) } - return recordLength; + return recordLength } override def close() { if (in != null) { in.close() } } diff --git a/src/main/scala/shark/execution/SelectOperator.scala b/src/main/scala/shark/execution/SelectOperator.scala index bdbb16c0..bcca156a 100755 --- a/src/main/scala/shark/execution/SelectOperator.scala +++ b/src/main/scala/shark/execution/SelectOperator.scala @@ -21,7 +21,6 @@ import scala.collection.JavaConversions._ import scala.reflect.BeanProperty import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} -import org.apache.hadoop.hive.ql.exec.{SelectOperator => HiveSelectOperator} import org.apache.hadoop.hive.ql.plan.SelectDesc import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 161a586a..b6f58f88 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -22,14 +22,14 @@ import java.util.{List => JavaList, Map => JavaMap} import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.{Context, DriverContext} -import org.apache.hadoop.hive.ql.exec.{Task => HiveTask, TaskExecutionException} +import org.apache.hadoop.hive.ql.exec.{Task => HiveTask} import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.plan.api.StageType import org.apache.spark.rdd.EmptyRDD -import shark.{LogHelper, SharkConfVars, SharkEnv} +import shark.{LogHelper, SharkEnv} import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} diff --git a/src/main/scala/shark/execution/SharkExplainTask.scala b/src/main/scala/shark/execution/SharkExplainTask.scala index 5d69e3f6..10fcd6f3 100755 --- a/src/main/scala/shark/execution/SharkExplainTask.scala +++ b/src/main/scala/shark/execution/SharkExplainTask.scala @@ -18,17 +18,15 @@ package shark.execution import java.io.PrintStream -import java.lang.reflect.Method -import java.util.{Arrays, HashSet, List => JavaList} +import java.util.{List => JavaList} import scala.collection.JavaConversions._ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.exec.{ExplainTask, Task} import org.apache.hadoop.hive.ql.{Context, DriverContext, QueryPlan} -import org.apache.hadoop.hive.ql.plan.{Explain, ExplainWork} -import org.apache.hadoop.hive.ql.plan.api.StageType +import org.apache.hadoop.hive.ql.exec.{ExplainTask, Task} +import org.apache.hadoop.hive.ql.plan.ExplainWork import org.apache.hadoop.util.StringUtils import shark.LogHelper @@ -89,7 +87,7 @@ class SharkExplainTask extends Task[SharkExplainWork] with java.io.Serializable case e: Exception => { console.printError("Failed with exception " + e.getMessage(), "\n" + StringUtils.stringifyException(e)) - throw(e) + throw e 1 } } diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 22836562..40cce0d9 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -17,21 +17,15 @@ package shark.execution -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.PathFilter -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} -import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.SerializableWritable -import shark.{LogHelper, SharkConfVars, SharkEnv} +import shark.{LogHelper, SharkEnv} import shark.api.QueryExecutionException -import shark.execution.optimization.ColumnPruner import shark.execution.serialization.JavaSerializer import shark.memstore2.{MemoryMetadataManager, TablePartition, TablePartitionStats} import shark.tachyon.TachyonException @@ -69,7 +63,7 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { // True if stats for the target table is missing from the Shark metastore, and should be fetched // and deserialized from Tachyon's metastore. This can happen if that table was created in a // previous Shark session, since Shark's metastore is not persistent. - var shouldFetchStatsFromTachyon = SharkEnv.memoryMetadataManager.getStats( + val shouldFetchStatsFromTachyon = SharkEnv.memoryMetadataManager.getStats( _databaseName, _tableName).isEmpty if (shouldFetchStatsFromTachyon) { val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) @@ -146,14 +140,14 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { if (iter.hasNext) { // Map each tuple to a row object val rowWithPartArr = new Array[Object](2) - val tablePartition = iter.next.asInstanceOf[TablePartition] + val tablePartition: TablePartition = iter.next() tablePartition.iterator.map { value => rowWithPartArr.update(0, value.asInstanceOf[Object]) rowWithPartArr.update(1, partValues) rowWithPartArr.asInstanceOf[Object] } } else { - Iterator() + Iterator.empty } } } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index bca3ddc8..1e035aca 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -37,7 +37,7 @@ import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.optimization.ColumnPruner -import shark.memstore2.{CacheType, MemoryMetadataManager, TablePartition, TablePartitionStats} +import shark.memstore2.{CacheType, TablePartition, TablePartitionStats} /** @@ -145,9 +145,9 @@ class TableScanOperator extends TopOperator[TableScanDesc] { val columnsUsed = new ColumnPruner(this, table).columnsUsed SharkEnv.tachyonUtil.pushDownColumnPruning(rdd, columnsUsed) - val shouldPrune = (SharkConfVars.getBoolVar(localHConf, SharkConfVars.MAP_PRUNING) && - childOperators(0).isInstanceOf[FilterOperator] && - indexToStats.size == rdd.partitions.size) + val shouldPrune = SharkConfVars.getBoolVar(localHConf, SharkConfVars.MAP_PRUNING) && + childOperators(0).isInstanceOf[FilterOperator] && + indexToStats.size == rdd.partitions.size val prunedRdd: RDD[_] = if (shouldPrune) { val startTime = System.currentTimeMillis @@ -184,10 +184,10 @@ class TableScanOperator extends TopOperator[TableScanDesc] { return prunedRdd.mapPartitions { iter => if (iter.hasNext) { - val tablePartition = iter.next.asInstanceOf[TablePartition] + val tablePartition = iter.next().asInstanceOf[TablePartition] tablePartition.prunedIterator(columnsUsed) } else { - Iterator() + Iterator.empty } } } @@ -232,25 +232,25 @@ object TableScanOperator extends LogHelper { val rowSchema = hiveTableScanOp.getSchema if (rowSchema != null) { // Add column names to the HiveConf. - var columnNames = new StringBuilder() + val columnNames = new StringBuilder for (columnInfo <- rowSchema.getSignature()) { - if (columnNames.length() > 0) { + if (columnNames.length > 0) { columnNames.append(",") } columnNames.append(columnInfo.getInternalName()) } - val columnNamesString = columnNames.toString(); - hiveConf.set(Constants.LIST_COLUMNS, columnNamesString); + val columnNamesString = columnNames.toString() + hiveConf.set(Constants.LIST_COLUMNS, columnNamesString) // Add column types to the HiveConf. - var columnTypes = new StringBuilder() + val columnTypes = new StringBuilder for (columnInfo <- rowSchema.getSignature()) { - if (columnTypes.length() > 0) { + if (columnTypes.length > 0) { columnTypes.append(",") } columnTypes.append(columnInfo.getType().getTypeName()) } - val columnTypesString = columnTypes.toString + val columnTypesString = columnTypes.toString() hiveConf.set(Constants.LIST_COLUMN_TYPES, columnTypesString) } diff --git a/src/main/scala/shark/execution/UDTFOperator.scala b/src/main/scala/shark/execution/UDTFOperator.scala index 17a39cd5..5782f370 100755 --- a/src/main/scala/shark/execution/UDTFOperator.scala +++ b/src/main/scala/shark/execution/UDTFOperator.scala @@ -23,11 +23,11 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.BeanProperty -import org.apache.hadoop.hive.ql.exec.{UDTFOperator => HiveUDTFOperator} import org.apache.hadoop.hive.ql.plan.UDTFDesc import org.apache.hadoop.hive.ql.udf.generic.Collector -import org.apache.hadoop.hive.serde2.objectinspector.{ ObjectInspector, - StandardStructObjectInspector, StructField, StructObjectInspector } +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.StructField class UDTFOperator extends UnaryOperator[UDTFDesc] { diff --git a/src/main/scala/shark/execution/UnionOperator.scala b/src/main/scala/shark/execution/UnionOperator.scala index 7df74d9e..84470f65 100755 --- a/src/main/scala/shark/execution/UnionOperator.scala +++ b/src/main/scala/shark/execution/UnionOperator.scala @@ -19,16 +19,13 @@ package shark.execution import java.util.{ArrayList, List => JavaList} -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.reflect.BeanProperty -import org.apache.hadoop.hive.ql.exec.{UnionOperator => HiveUnionOperator} import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ReturnObjectInspectorResolver import org.apache.hadoop.hive.ql.plan.UnionDesc import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory -import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.StructField import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector diff --git a/src/main/scala/shark/execution/optimization/ColumnPruner.scala b/src/main/scala/shark/execution/optimization/ColumnPruner.scala index f116575e..d348fa3e 100644 --- a/src/main/scala/shark/execution/optimization/ColumnPruner.scala +++ b/src/main/scala/shark/execution/optimization/ColumnPruner.scala @@ -20,7 +20,7 @@ package shark.execution.optimization import java.util.BitSet import java.util.{List => JList} -import scala.collection.JavaConversions.{asScalaBuffer, bufferAsJavaList, collectionAsScalaIterable} +import scala.collection.JavaConversions.{asScalaBuffer, collectionAsScalaIterable} import scala.collection.mutable.{Set, HashSet} import org.apache.hadoop.hive.ql.exec.GroupByPreShuffleOperator @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.ql.plan.{FilterDesc, MapJoinDesc, ReduceSinkDesc} import shark.execution.{FilterOperator, JoinOperator, MapJoinOperator, Operator, ReduceSinkOperator, SelectOperator, TopOperator} -import shark.memstore2.{ColumnarStruct, TablePartitionIterator} class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends Serializable { @@ -39,8 +38,8 @@ class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends val columnsUsed: BitSet = { val colsToKeep = computeColumnsToKeep() val allColumns = tbl.getAllCols().map(x => x.getName()) - var b = new BitSet() - for (i <- Range(0, allColumns.size()) if (colsToKeep.contains(allColumns(i)))) { + val b = new BitSet() + for (i <- Range(0, allColumns.size) if colsToKeep.contains(allColumns(i))) { b.set(i, true) } b @@ -55,11 +54,13 @@ class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends /** * Computes the column names that are referenced in the Query */ - private def computeColumnsToKeep(op: Operator[_], - cols: HashSet[String], parentOp: Operator[_] = null): Unit = { + private def computeColumnsToKeep( + op: Operator[_], cols: HashSet[String], parentOp: Operator[_] = null): Unit = { + def nullGuard[T](s: JList[T]): Seq[T] = { if (s == null) Seq[T]() else s } + op match { case selOp: SelectOperator => { val cnf:SelectDesc = selOp.getConf @@ -84,7 +85,7 @@ class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends if (cnf != null) { val keyEvals = nullGuard(cnf.getKeyCols) val valEvals = nullGuard(cnf.getValueCols) - val evals = (HashSet() ++ keyEvals ++ valEvals) + val evals = HashSet() ++ keyEvals ++ valEvals cols ++= evals.flatMap(x => nullGuard(x.getCols)) } } @@ -93,7 +94,7 @@ class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends if (cnf != null) { val keyEvals = cnf.getKeys.values val valEvals = cnf.getExprs.values - val evals = (HashSet() ++ keyEvals ++ valEvals) + val evals = HashSet() ++ keyEvals ++ valEvals cols ++= evals.flatMap(x => x).flatMap(x => nullGuard(x.getCols)) } } diff --git a/src/main/scala/shark/execution/serialization/HiveStructDeserializer.scala b/src/main/scala/shark/execution/serialization/HiveStructDeserializer.scala index 9589a1e9..2a54fbf3 100644 --- a/src/main/scala/shark/execution/serialization/HiveStructDeserializer.scala +++ b/src/main/scala/shark/execution/serialization/HiveStructDeserializer.scala @@ -23,8 +23,6 @@ package org.apache.hadoop.hive.serde2.binarysortable import java.io.IOException import java.util.{ArrayList => JArrayList} -import scala.collection.JavaConversions._ - import org.apache.hadoop.hive.serde2.SerDeException import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoUtils} diff --git a/src/main/scala/shark/execution/serialization/HiveStructSerializer.scala b/src/main/scala/shark/execution/serialization/HiveStructSerializer.scala index 5b99ba09..1f5544fb 100644 --- a/src/main/scala/shark/execution/serialization/HiveStructSerializer.scala +++ b/src/main/scala/shark/execution/serialization/HiveStructSerializer.scala @@ -22,8 +22,6 @@ package org.apache.hadoop.hive.serde2.binarysortable import java.util.{List => JList} -import scala.collection.JavaConversions._ - import org.apache.hadoop.hive.serde2.objectinspector.{StructField, StructObjectInspector} diff --git a/src/main/scala/shark/execution/serialization/XmlSerializer.scala b/src/main/scala/shark/execution/serialization/XmlSerializer.scala index 4c63efab..a533c812 100644 --- a/src/main/scala/shark/execution/serialization/XmlSerializer.scala +++ b/src/main/scala/shark/execution/serialization/XmlSerializer.scala @@ -17,8 +17,8 @@ package shark.execution.serialization -import java.beans.{XMLDecoder, XMLEncoder, PersistenceDelegate} -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectOutput, ObjectInput} +import java.beans.{XMLDecoder, XMLEncoder} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import com.ning.compress.lzf.{LZFEncoder, LZFDecoder} @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities.EnumDelegate import org.apache.hadoop.hive.ql.plan.GroupByDesc import org.apache.hadoop.hive.ql.plan.PlanUtils.ExpressionTypes -import shark.{SharkConfVars, SharkEnvSlave} +import shark.SharkConfVars /** diff --git a/src/main/scala/shark/memstore2/CachePolicy.scala b/src/main/scala/shark/memstore2/CachePolicy.scala index a5b2cfcd..27e29ff6 100644 --- a/src/main/scala/shark/memstore2/CachePolicy.scala +++ b/src/main/scala/shark/memstore2/CachePolicy.scala @@ -23,8 +23,6 @@ import java.util.Map.Entry import scala.collection.JavaConversions._ -import org.apache.spark.rdd.RDD - /** * An general interface for pluggable cache eviction policies in Shark. diff --git a/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala b/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala index 5a943443..67a99612 100644 --- a/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala +++ b/src/main/scala/shark/memstore2/ColumnarStructObjectInspector.scala @@ -27,8 +27,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo -import shark.SharkConfVars - class ColumnarStructObjectInspector(fields: JList[StructField]) extends StructObjectInspector { diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index ca64fd27..34a1a048 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -18,7 +18,7 @@ package shark.memstore2 import java.util.concurrent.ConcurrentHashMap -import java.util.{HashMap => JavaHashMap, Map => JavaMap} +import java.util.{Map => JavaMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ConcurrentMap @@ -28,7 +28,6 @@ import org.apache.spark.storage.StorageLevel import shark.execution.RDDUtils import shark.SharkConfVars -import shark.SharkEnv class MemoryMetadataManager { @@ -58,8 +57,8 @@ class MemoryMetadataManager { def isHivePartitioned(databaseName: String, tableName: String): Boolean = { val tableKey = makeTableKey(databaseName, tableName) _keyToTable.get(tableKey) match { - case Some(table) => return table.isInstanceOf[PartitionedMemoryTable] - case None => return false + case Some(table) => table.isInstanceOf[PartitionedMemoryTable] + case None => false } } @@ -74,9 +73,9 @@ class MemoryMetadataManager { preferredStorageLevel: StorageLevel ): MemoryTable = { val tableKey = makeTableKey(databaseName, tableName) - var newTable = new MemoryTable(tableKey, cacheMode, preferredStorageLevel) + val newTable = new MemoryTable(tableKey, cacheMode, preferredStorageLevel) _keyToTable.put(tableKey, newTable) - return newTable + newTable } def createPartitionedMemoryTable( @@ -87,7 +86,7 @@ class MemoryMetadataManager { tblProps: JavaMap[String, String] ): PartitionedMemoryTable = { val tableKey = makeTableKey(databaseName, tableName) - var newTable = new PartitionedMemoryTable(tableKey, cacheMode, preferredStorageLevel) + val newTable = new PartitionedMemoryTable(tableKey, cacheMode, preferredStorageLevel) // Determine the cache policy to use and read any user-specified cache settings. val cachePolicyStr = tblProps.getOrElse(SharkConfVars.CACHE_POLICY.varname, SharkConfVars.CACHE_POLICY.defaultVal) @@ -96,7 +95,7 @@ class MemoryMetadataManager { newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize) _keyToTable.put(tableKey, newTable) - return newTable + newTable } def getTable(databaseName: String, tableName: String): Option[Table] = { @@ -143,18 +142,17 @@ class MemoryMetadataManager { * Used to drop a table from the Spark in-memory cache and/or disk. All metadata tracked by Shark * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) is deleted as well. * - * @param key Name of the table to drop. * @return Option::isEmpty() is true of there is no MemoryTable (and RDD) corresponding to 'key' * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. */ - def removeTable(databaseName:String, tableName: String): Option[RDD[_]] = { + def removeTable(databaseName: String, tableName: String): Option[RDD[_]] = { val tableKey = makeTableKey(databaseName, tableName) // Remove MemoryTable's entry from Shark metadata. _keyToStats.remove(tableKey) val tableValue: Option[Table] = _keyToTable.remove(tableKey) - return tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) + tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) } /** @@ -189,7 +187,7 @@ object MemoryMetadataManager { } else { unpersistedRDD = Some(RDDUtils.unpersistRDD(table.asInstanceOf[MemoryTable].tableRDD)) } - return unpersistedRDD + unpersistedRDD } /** @@ -199,8 +197,7 @@ object MemoryMetadataManager { def makeHivePartitionKeyStr( partitionCols: Seq[String], partColToValue: JavaMap[String, String]): String = { - val keyStr = partitionCols.map(col => "%s=%s".format(col, partColToValue(col))).mkString("/") - return keyStr + partitionCols.map(col => "%s=%s".format(col, partColToValue(col))).mkString("/") } /** Return a StorageLevel corresponding to its String name. */ diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 020a112f..cb61d47b 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -17,8 +17,6 @@ package shark.memstore2 -import shark.execution.RDDUtils - import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index fb63596d..c0c43c2c 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -54,7 +54,7 @@ class PartitionedMemoryTable( // A map from the Hive-partition key to the RDD that contains contents of that partition. // The conventional string format for the partition key, 'col1=value1/col2=value2/...', can be // computed using MemoryMetadataManager#makeHivePartitionKeyStr() - private var _keyToPartitions: ConcurrentMap[String, RDDValue] = + private val _keyToPartitions: ConcurrentMap[String, RDDValue] = new ConcurrentJavaHashMap[String, RDDValue]() // The eviction policy for this table's cached Hive-partitions. An example of how this @@ -68,7 +68,7 @@ class PartitionedMemoryTable( def getPartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddValueOpt: Option[RDDValue] = _keyToPartitions.get(partitionKey) if (rddValueOpt.isDefined) _cachePolicy.notifyGet(partitionKey) - return rddValueOpt.map(_.rdd) + rddValueOpt.map(_.rdd) } def putPartition( @@ -76,18 +76,18 @@ class PartitionedMemoryTable( newRDD: RDD[TablePartition], isUpdate: Boolean = false): Option[RDD[TablePartition]] = { val rddValueOpt = _keyToPartitions.get(partitionKey) - var prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) + val prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) val newRDDValue = new RDDValue(newRDD) _keyToPartitions.put(partitionKey, newRDDValue) _cachePolicy.notifyPut(partitionKey, newRDDValue) - return prevRDD + prevRDD } def updatePartition( partitionKey: String, updatedRDD: RDD[TablePartition]): Option[RDD[TablePartition]] = { val rddValueOpt = _keyToPartitions.get(partitionKey) - var prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) + val prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) if (rddValueOpt.isDefined) { // This is an update of an old value, so update the RDDValue's `rdd` entry. // Don't notify the `_cachePolicy`. Assumes that getPartition() has already been called to @@ -96,23 +96,24 @@ class PartitionedMemoryTable( val updatedRDDValue = rddValueOpt.get updatedRDDValue.rdd = updatedRDD } - return prevRDD + prevRDD } def removePartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) - if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey) - return rddRemoved.map(_.rdd) + if (rddRemoved.isDefined) { + _cachePolicy.notifyRemove(partitionKey) + } + rddRemoved.map(_.rdd) } def setPartitionCachePolicy(cachePolicyStr: String, fallbackMaxSize: Int) { // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. - val loadFunc: String => RDDValue = - (partitionKey: String) => { - val rddValue = _keyToPartitions.get(partitionKey).get - rddValue.rdd.persist(preferredStorageLevel) - rddValue - } + val loadFunc: String => RDDValue = (partitionKey: String) => { + val rddValue = _keyToPartitions.get(partitionKey).get + rddValue.rdd.persist(preferredStorageLevel) + rddValue + } // The evictionFunc will unpersist the RDD. val evictionFunc: (String, RDDValue) => Unit = (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) @@ -125,7 +126,7 @@ class PartitionedMemoryTable( /** Returns an immutable view of the String->RDD mapping to external callers */ def keyToPartitions: collection.immutable.Map[String, RDD[TablePartition]] = { - return _keyToPartitions.mapValues(_.rdd).toMap + _keyToPartitions.mapValues(_.rdd).toMap } } diff --git a/src/main/scala/shark/memstore2/TablePartitionIterator.scala b/src/main/scala/shark/memstore2/TablePartitionIterator.scala index 07312fc1..947cdd22 100644 --- a/src/main/scala/shark/memstore2/TablePartitionIterator.scala +++ b/src/main/scala/shark/memstore2/TablePartitionIterator.scala @@ -44,13 +44,13 @@ class TablePartitionIterator( private var _position: Long = 0 - def hasNext(): Boolean = _position < numRows + def hasNext: Boolean = _position < numRows def next(): ColumnarStruct = { _position += 1 var i = columnUsed.nextSetBit(0) while (i > -1) { - columnIterators(i).next + columnIterators(i).next() i = columnUsed.nextSetBit(i + 1) } _struct diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 8b1e5625..f45fea12 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -27,11 +27,8 @@ import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer import org.apache.hadoop.hive.ql.parse.HiveParser import org.apache.hadoop.hive.ql.plan.DDLWork -import org.apache.spark.rdd.{UnionRDD, RDD} - import shark.execution.SharkDDLWork import shark.{LogHelper, SharkEnv} -import shark.memstore2.MemoryMetadataManager class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) with LogHelper { diff --git a/src/main/scala/shark/parse/SharkExplainSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkExplainSemanticAnalyzer.scala index c8d69322..e139ac27 100755 --- a/src/main/scala/shark/parse/SharkExplainSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkExplainSemanticAnalyzer.scala @@ -19,13 +19,11 @@ package shark.parse import java.io.Serializable import java.util.ArrayList -import java.util.List import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec._ import org.apache.hadoop.hive.ql.parse._ -import org.apache.hadoop.hive.ql.plan.ExplainWork import shark.execution.SharkExplainWork diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 9aa2bdcb..3f984848 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -17,7 +17,6 @@ package shark.parse -import java.lang.reflect.Method import java.util.ArrayList import java.util.{List => JavaList} import java.util.{Map => JavaMap} @@ -37,11 +36,10 @@ import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.spark.storage.StorageLevel -import shark.{CachedTableRecovery, LogHelper, SharkConfVars, SharkEnv, Utils} -import shark.execution.{HiveDesc, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator, - SharkDDLWork, SparkWork, TerminalOperator} +import shark.{CachedTableRecovery, LogHelper, SharkConfVars, SharkEnv} +import shark.execution.{HiveDesc, Operator, OperatorFactory, ReduceSinkOperator, SharkDDLWork, + SparkWork, TerminalOperator} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} @@ -219,7 +217,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with preferredStorageLevel, _resSchema.size, /* numColumns */ new String, /* hivePartitionKey */ - qb.getCacheModeForCreateTable, + qb.getCacheModeForCreateTable(), false /* useUnionRDD */) } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala index 5ef6efec..c34052bb 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzerFactory.scala @@ -30,17 +30,17 @@ object SharkSemanticAnalyzerFactory { * Return a semantic analyzer for the given ASTNode. */ def get(conf: HiveConf, tree:ASTNode): BaseSemanticAnalyzer = { - val baseSem = SemanticAnalyzerFactory.get(conf, tree) + val explainMode = SharkConfVars.getVar(conf, SharkConfVars.EXPLAIN_MODE) == "shark" - if (baseSem.isInstanceOf[SemanticAnalyzer]) { - new SharkSemanticAnalyzer(conf) - } else if (baseSem.isInstanceOf[ExplainSemanticAnalyzer] && - SharkConfVars.getVar(conf, SharkConfVars.EXPLAIN_MODE) == "shark") { - new SharkExplainSemanticAnalyzer(conf) - } else if (baseSem.isInstanceOf[DDLSemanticAnalyzer]) { - new SharkDDLSemanticAnalyzer(conf) - } else { - baseSem + SemanticAnalyzerFactory.get(conf, tree) match { + case _: SemanticAnalyzer => + new SharkSemanticAnalyzer(conf) + case _: ExplainSemanticAnalyzer if explainMode => + new SharkExplainSemanticAnalyzer(conf) + case _: DDLSemanticAnalyzer => + new SharkDDLSemanticAnalyzer(conf) + case sem: BaseSemanticAnalyzer => + sem } } } diff --git a/src/main/scala/shark/util/BloomFilter.scala b/src/main/scala/shark/util/BloomFilter.scala index 8183a6a9..ce4d46e0 100644 --- a/src/main/scala/shark/util/BloomFilter.scala +++ b/src/main/scala/shark/util/BloomFilter.scala @@ -19,8 +19,9 @@ package shark.util import java.util.BitSet import java.nio.charset.Charset -import scala.math._ -import com.google.common.primitives.Bytes + +import scala.math.{ceil, log} + import com.google.common.primitives.Ints import com.google.common.primitives.Longs @@ -33,13 +34,12 @@ import com.google.common.primitives.Longs * @param expectedSize is the number of elements to be contained in the filter. * @param numHashes is the number of hash functions. * @author Ram Sriharsha (harshars at yahoo-inc dot com) - * @date 07/07/2013 */ class BloomFilter(numBitsPerElement: Double, expectedSize: Int, numHashes: Int) - extends AnyRef with Serializable{ + extends AnyRef with Serializable { val SEED = System.getProperty("shark.bloomfilter.seed","1234567890").toInt - val bitSetSize = ceil(numBitsPerElement * expectedSize).toInt + val bitSetSize = math.ceil(numBitsPerElement * expectedSize).toInt val bitSet = new BitSet(bitSetSize) /** @@ -68,7 +68,7 @@ class BloomFilter(numBitsPerElement: Double, expectedSize: Int, numHashes: Int) * Optimization to allow reusing the same input buffer by specifying * the length of the buffer that contains the bytes to be hashed. * @param data is the bytes to be hashed. - * @param length is the length of the buffer to examine. + * @param len is the length of the buffer to examine. */ def add(data: Array[Byte], len: Int) { val hashes = hash(data, numHashes, len) @@ -113,7 +113,7 @@ class BloomFilter(numBitsPerElement: Double, expectedSize: Int, numHashes: Int) * Optimization to allow reusing the same input buffer by specifying * the length of the buffer that contains the bytes to be hashed. * @param data is the bytes to be hashed. - * @param length is the length of the buffer to examine. + * @param len is the length of the buffer to examine. * @return true with some false positive probability and false if the * bytes is not contained in the bloom filter. */ diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 5a9e686e..447b85b9 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -75,7 +75,7 @@ private[shark] object HiveUtils { new JavaHashSet[WriteEntity], createTbleDesc) val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) - return (taskExecutionStatus == 0) + taskExecutionStatus == 0 } def dropTableInHive( @@ -92,7 +92,7 @@ private[shark] object HiveUtils { new JavaHashSet[WriteEntity], dropTblDesc) val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) - return (taskExecutionStatus == 0) + taskExecutionStatus == 0 } def executeDDLTaskDirectly(ddlWork: DDLWork): Int = { @@ -101,6 +101,6 @@ private[shark] object HiveUtils { task.setWork(ddlWork) // Hive returns 0 if the create table command is executed successfully. - return task.execute(null) + task.execute(null) } } diff --git a/src/main/scala/shark/util/MurmurHash3_x86_128.scala b/src/main/scala/shark/util/MurmurHash3_x86_128.scala index 47e9220e..a1743a0a 100644 --- a/src/main/scala/shark/util/MurmurHash3_x86_128.scala +++ b/src/main/scala/shark/util/MurmurHash3_x86_128.scala @@ -18,7 +18,6 @@ package shark.util import java.lang.Integer.{ rotateLeft => rotl } -import scala.math._ /** *

The MurmurHash3_x86_128(...) is a fast, non-cryptographic, 128-bit hash @@ -126,7 +125,7 @@ object MurmurHash3_x86_128 { * @param seed is the seed for the murmurhash algorithm. * @param length is the length of the buffer to use for hashing. * @param results is the output buffer to store the four ints that are returned, - * should have size atleast 4. + * should have size at least 4. */ @inline final def hash(data: Array[Byte], seed: Int, length: Int, results: Array[Int]): Unit = { From 0ead2cc501f62470b01acb4b5b8d6e91dca59335 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 2 Nov 2013 12:35:22 -0700 Subject: [PATCH 200/331] Force user specified hadoop dependency. --- project/SharkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index c9f32bb9..0ef1e8a4 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -113,7 +113,7 @@ object SharkBuild extends Build { "org.apache.spark" %% "spark-core" % SPARK_VERSION, "org.apache.spark" %% "spark-repl" % SPARK_VERSION, "com.google.guava" % "guava" % "14.0.1", - "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) force(), // See https://code.google.com/p/guava-libraries/issues/detail?id=1095 "com.google.code.findbugs" % "jsr305" % "1.3.+", From de076c9c842ae7ab18ba89384863346afc79d244 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 2 Nov 2013 23:45:16 -0700 Subject: [PATCH 201/331] Addressed Harvey's comment. --- src/main/scala/shark/execution/MapSplitPruning.scala | 8 ++++++-- .../shark/execution/serialization/ShuffleSerializer.scala | 2 +- src/main/scala/shark/util/BloomFilter.scala | 2 +- src/main/scala/shark/util/MurmurHash3_x86_128.scala | 2 +- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index f89b8676..974e6cae 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -142,7 +142,9 @@ object MapSplitPruning { left.asInstanceOf[ExprNodeColumnEvaluator] } else if (right.isInstanceOf[ExprNodeColumnEvaluator]) { right.asInstanceOf[ExprNodeColumnEvaluator] - } else null + } else { + null + } // Try to get the constant value. val constEval: ExprNodeConstantEvaluator = @@ -150,7 +152,9 @@ object MapSplitPruning { left.asInstanceOf[ExprNodeConstantEvaluator] } else if (right.isInstanceOf[ExprNodeConstantEvaluator]) { right.asInstanceOf[ExprNodeConstantEvaluator] - } else null + } else { + null + } if (columnEval != null && constEval != null) { // We can prune the partition only if it is a predicate of form diff --git a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala index defdae23..ae91086b 100644 --- a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala +++ b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import org.apache.hadoop.io.BytesWritable import org.apache.spark.serializer.DeserializationStream -import org.apache.spark.serializer.{Serializer, SerializerInstance, SerializationStream} +import org.apache.spark.serializer.{SerializationStream, Serializer, SerializerInstance} import shark.execution.{ReduceKey, ReduceKeyReduceSide} diff --git a/src/main/scala/shark/util/BloomFilter.scala b/src/main/scala/shark/util/BloomFilter.scala index 8183a6a9..82a91fbe 100644 --- a/src/main/scala/shark/util/BloomFilter.scala +++ b/src/main/scala/shark/util/BloomFilter.scala @@ -115,7 +115,7 @@ class BloomFilter(numBitsPerElement: Double, expectedSize: Int, numHashes: Int) * @param data is the bytes to be hashed. * @param length is the length of the buffer to examine. * @return true with some false positive probability and false if the - * bytes is not contained in the bloom filter. + * bytes is not contained in the bloom filter. */ def contains(data: Array[Byte], len: Int): Boolean = { !hash(data,numHashes, len).exists { diff --git a/src/main/scala/shark/util/MurmurHash3_x86_128.scala b/src/main/scala/shark/util/MurmurHash3_x86_128.scala index 47e9220e..799c7135 100644 --- a/src/main/scala/shark/util/MurmurHash3_x86_128.scala +++ b/src/main/scala/shark/util/MurmurHash3_x86_128.scala @@ -194,7 +194,7 @@ object MurmurHash3_x86_128 { * @param rem is the remainder of the byte array to examine. */ @inline final def getInt(data: Array[Byte], index: Int, rem: Int): Int = { - rem match { + rem match { case 3 => data(index) << 24 | (data(index + 1) & 0xFF) << 16 | (data(index + 2) & 0xFF) << 8 From 90082e0c946bb66dcb21344247ae49b94ffbc900 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 3 Nov 2013 13:17:49 -0800 Subject: [PATCH 202/331] Add a util method that directly launches a task which alters the SerDe property of a table or table partition. --- src/main/scala/shark/util/HiveUtils.scala | 50 ++++++++++++++++++----- 1 file changed, 40 insertions(+), 10 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 5a9e686e..6931a82a 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -17,18 +17,22 @@ package shark.util -import java.util.{ArrayList => JavaArrayList, HashSet => JavaHashSet} +import java.util.{ArrayList => JavaArrayList, HashMap => JavaHashMap, HashSet => JavaHashSet} + import scala.collection.JavaConversions._ +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} -import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.ql.plan.AlterTableDesc +import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes + import shark.api.{DataType, DataTypes} -import org.apache.hadoop.hive.ql.exec.DDLTask -import org.apache.hadoop.hive.conf.HiveConf private[shark] object HiveUtils { @@ -70,16 +74,15 @@ private[shark] object HiveUtils { createTbleDesc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) createTbleDesc.setNumBuckets(-1) - // Execute the create table against the metastore. + // Execute the create table against the Hive metastore. val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], createTbleDesc) val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) - return (taskExecutionStatus == 0) + taskExecutionStatus == 0 } - def dropTableInHive( - tableName: String): Boolean = { + def dropTableInHive(tableName: String): Boolean = { // Setup the drop table descriptor with necessary information. val dropTblDesc = new DropTableDesc( tableName, @@ -92,7 +95,34 @@ private[shark] object HiveUtils { new JavaHashSet[WriteEntity], dropTblDesc) val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) - return (taskExecutionStatus == 0) + taskExecutionStatus == 0 + } + + /** + * Directly executes a Hive DDLTask that changes the SerDe property for the table (or table + * partition) entry in the Hive metastore. Returns `true` if successful. + * + * @tableName Name of table being altered. + * @partitionSpec Map of (partition col, partition key) pairs for which the SerDe is being + * altered. Partition values must always be given if the table is Hive partitioned and must + * be empty if the table isn't Hive partitioned. + * @serDeName Class name of new SerDe to use. + */ + def alterSerdeInHive( + tableName: String, + partitionSpec: JavaHashMap[String, String], + serDeName: String): Boolean = { + val alterTableDesc = new AlterTableDesc(AlterTableTypes.ADDSERDE) + alterTableDesc.setOldName(tableName) + alterTableDesc.setSerdeName(serDeName) + alterTableDesc.setPartSpec(partitionSpec) + + // Execute the SerDe change against the Hive metastore. + val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], + new JavaHashSet[WriteEntity], + alterTableDesc) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + taskExecutionStatus == 0 } def executeDDLTaskDirectly(ddlWork: DDLWork): Int = { @@ -101,6 +131,6 @@ private[shark] object HiveUtils { task.setWork(ddlWork) // Hive returns 0 if the create table command is executed successfully. - return task.execute(null) + task.execute(null) } } From 439633c5b69a11cf9f6fef7b3064371715ac71c5 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 3 Nov 2013 14:24:46 -0800 Subject: [PATCH 203/331] Tests for HiveUtils methods that alter table/partition SerDe. --- src/main/scala/shark/util/HiveUtils.scala | 10 ++-- src/test/scala/shark/SQLSuite.scala | 60 +++++++++++++++++++++++ 2 files changed, 65 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 9c319887..a8dfee50 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} import org.apache.hadoop.hive.ql.plan.AlterTableDesc import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} - +import shark.SharkContext import shark.api.{DataType, DataTypes} @@ -103,8 +103,7 @@ private[shark] object HiveUtils { * * @tableName Name of table being altered. * @partitionSpec Map of (partition col, partition key) pairs for which the SerDe is being - * altered. Partition values must always be given if the table is Hive partitioned and must - * be empty if the table isn't Hive partitioned. + * altered. NULL if the table isn't Hive-partitioned. * @serDeName Class name of new SerDe to use. */ def alterSerdeInHive( @@ -126,10 +125,11 @@ private[shark] object HiveUtils { def executeDDLTaskDirectly(ddlWork: DDLWork): Int = { val task = new DDLTask() - task.initialize(new HiveConf, null, null) + task.initialize(new HiveConf, null /* queryPlan */, null /* ctx: DriverContext */) + task.setWork(ddlWork) // Hive returns 0 if the create table command is executed successfully. - task.execute(null) + task.execute(null /* driverContext */) } } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index d3832478..516adb7f 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -17,14 +17,20 @@ package shark +import java.util.{HashMap => JavaHashMap} + import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.apache.hadoop.hive.ql.metadata.Hive + import org.apache.spark.rdd.UnionRDD import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.PartitionedMemoryTable +import shark.util.HiveUtils + class SQLSuite extends FunSuite with BeforeAndAfterAll { @@ -860,4 +866,58 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val e = intercept[QueryExecutionException] { sc.sql2rdd("asdfasdfasdfasdf") } e.getMessage.contains("semantic") } + + ////////////////////////////////////////////////////////////////////////////// + // HiveUtils methods that modify Hive metastore + // TODO(harvey): Tests for methods that directly create/drop tables. + ////////////////////////////////////////////////////////////////////////////// + + test("HiveUtils: directly alter table's SerDe") { + def getTableSerDeName(tableName: String): String = { + val hiveTable = Hive.get().getTable(tableName) + hiveTable.getDeserializer.getClass.getName + } + + sc.runSql("drop table if exists alter_table_serde") + sc.runSql("create table alter_table_serde (key int, value string)") + val tableName = "alter_table_serde" + + val oldSerDeName = getTableSerDeName(tableName) + val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName + + // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe. + HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, columnarSerDeName) + assert(getTableSerDeName(tableName) == columnarSerDeName) + + // Change the SerDe back to LazySimpleSerDe. + HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, oldSerDeName) + assert(getTableSerDeName(tableName) == oldSerDeName) + } + + test("HiveUtils: directly alter table partition's SerDe") { + def getPartitionSerDeName(tableName: String, partSpec: JavaHashMap[String, String]): String = { + // Get Hive metadata objects. + val metastore = Hive.get() + val table = metastore.getTable(tableName) + val partition = metastore.getPartition(table, partSpec, false /* forceCreate */) + partition.getDeserializer.getClass.getName + } + + sc.runSql("drop table if exists alter_part_serde") + sc.runSql("create table alter_part_serde (key int, value string) partitioned by (keypart int)") + sc.runSql("insert into table alter_part_serde partition (keypart = 1) select * from test") + val tableName = "alter_part_serde" + val partitionSpec = new JavaHashMap[String, String]() + partitionSpec.put("keypart", "1") + val oldSerDeName = getPartitionSerDeName(tableName, partitionSpec) + val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName + + // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe + HiveUtils.alterSerdeInHive(tableName, partitionSpec, columnarSerDeName) + assert(getPartitionSerDeName(tableName, partitionSpec) == columnarSerDeName) + + // Change the SerDe back to LazySimpleSerDe. + HiveUtils.alterSerdeInHive(tableName, partitionSpec, oldSerDeName) + assert(getPartitionSerDeName(tableName, partitionSpec) == oldSerDeName) + } } From a6fbbdfee3bc2e07ecae328044f516c006e6d5f5 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 3 Nov 2013 15:55:40 -0800 Subject: [PATCH 204/331] Take optional HiveConf (meant to reflect a session state) in HiveUtils metastore helpers. --- src/main/scala/shark/util/HiveUtils.scala | 19 +++++++++++-------- src/test/scala/shark/SQLSuite.scala | 14 ++++++++------ 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index a8dfee50..28939f92 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -58,7 +58,8 @@ private[shark] object HiveUtils { def createTableInHive( tableName: String, columnNames: Seq[String], - columnTypes: Seq[ClassManifest[_]]): Boolean = { + columnTypes: Seq[ClassManifest[_]], + hiveConf: HiveConf = new HiveConf): Boolean = { val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") } @@ -77,11 +78,11 @@ private[shark] object HiveUtils { val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], createTbleDesc) - val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork, hiveConf) taskExecutionStatus == 0 } - def dropTableInHive(tableName: String): Boolean = { + def dropTableInHive(tableName: String, hiveConf: HiveConf = new HiveConf): Boolean = { // Setup the drop table descriptor with necessary information. val dropTblDesc = new DropTableDesc( tableName, @@ -93,7 +94,7 @@ private[shark] object HiveUtils { val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], dropTblDesc) - val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork, hiveConf) taskExecutionStatus == 0 } @@ -105,11 +106,13 @@ private[shark] object HiveUtils { * @partitionSpec Map of (partition col, partition key) pairs for which the SerDe is being * altered. NULL if the table isn't Hive-partitioned. * @serDeName Class name of new SerDe to use. + * @hiveConf Configuration associated with the current SessionState. */ def alterSerdeInHive( tableName: String, partitionSpec: JavaHashMap[String, String], - serDeName: String): Boolean = { + serDeName: String, + hiveConf: HiveConf = new HiveConf): Boolean = { val alterTableDesc = new AlterTableDesc(AlterTableDesc.AlterTableTypes.ADDSERDE) alterTableDesc.setOldName(tableName) alterTableDesc.setSerdeName(serDeName) @@ -119,13 +122,13 @@ private[shark] object HiveUtils { val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], alterTableDesc) - val taskExecutionStatus = executeDDLTaskDirectly(ddlWork) + val taskExecutionStatus = executeDDLTaskDirectly(ddlWork, hiveConf) taskExecutionStatus == 0 } - def executeDDLTaskDirectly(ddlWork: DDLWork): Int = { + def executeDDLTaskDirectly(ddlWork: DDLWork, hiveConf: HiveConf): Int = { val task = new DDLTask() - task.initialize(new HiveConf, null /* queryPlan */, null /* ctx: DriverContext */) + task.initialize(hiveConf, null /* queryPlan */, null /* ctx: DriverContext */) task.setWork(ddlWork) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 516adb7f..85d671b1 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -874,30 +874,31 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test("HiveUtils: directly alter table's SerDe") { def getTableSerDeName(tableName: String): String = { - val hiveTable = Hive.get().getTable(tableName) + val hiveTable = Hive.get(SharkContext.hiveconf).getTable(tableName) hiveTable.getDeserializer.getClass.getName } sc.runSql("drop table if exists alter_table_serde") sc.runSql("create table alter_table_serde (key int, value string)") val tableName = "alter_table_serde" + val hiveConf = SharkContext.hiveconf val oldSerDeName = getTableSerDeName(tableName) val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe. - HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, columnarSerDeName) + HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, columnarSerDeName, hiveConf) assert(getTableSerDeName(tableName) == columnarSerDeName) // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, oldSerDeName) + HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, oldSerDeName, hiveConf) assert(getTableSerDeName(tableName) == oldSerDeName) } test("HiveUtils: directly alter table partition's SerDe") { def getPartitionSerDeName(tableName: String, partSpec: JavaHashMap[String, String]): String = { // Get Hive metadata objects. - val metastore = Hive.get() + val metastore = Hive.get(SharkContext.hiveconf) val table = metastore.getTable(tableName) val partition = metastore.getPartition(table, partSpec, false /* forceCreate */) partition.getDeserializer.getClass.getName @@ -907,17 +908,18 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("create table alter_part_serde (key int, value string) partitioned by (keypart int)") sc.runSql("insert into table alter_part_serde partition (keypart = 1) select * from test") val tableName = "alter_part_serde" + val hiveConf = SharkContext.hiveconf val partitionSpec = new JavaHashMap[String, String]() partitionSpec.put("keypart", "1") val oldSerDeName = getPartitionSerDeName(tableName, partitionSpec) val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe - HiveUtils.alterSerdeInHive(tableName, partitionSpec, columnarSerDeName) + HiveUtils.alterSerdeInHive(tableName, partitionSpec, columnarSerDeName, hiveConf) assert(getPartitionSerDeName(tableName, partitionSpec) == columnarSerDeName) // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive(tableName, partitionSpec, oldSerDeName) + HiveUtils.alterSerdeInHive(tableName, partitionSpec, oldSerDeName, hiveConf) assert(getPartitionSerDeName(tableName, partitionSpec) == oldSerDeName) } } From 89ebc447995c4b62130d2479ceeb417af5bbfa3c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 3 Nov 2013 16:22:29 -0800 Subject: [PATCH 205/331] A few small comments for SharkSemanticAnalyzer. --- src/main/scala/shark/parse/SharkSemanticAnalyzer.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 9aa2bdcb..edc51700 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -276,6 +276,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val mvTasks = new ArrayList[MoveTask]() val fileWork = getParseContext.getLoadFileWork + // Used for loading non-native tables. val tableWork = getParseContext.getLoadTableWork tableWork.foreach { ltd => mvTasks.add(TaskFactory.get( @@ -290,7 +291,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val dumpTable = db.newTable(qb.getTableDesc.getTableName) val wh = new Warehouse(conf) location = wh.getTablePath(db.getDatabase(dumpTable.getDbName()), dumpTable - .getTableName()).toString; + .getTableName()).toString; } catch { case e: HiveException => throw new SemanticException(e) case e: MetaException => throw new SemanticException(e) @@ -305,6 +306,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // The move task depends on all root tasks. In the case of multi outputs, // the moves are only started once all outputs are executed. + // Note: For a CTAS for a memory-only cached table, a MoveTask is still added as a child of + // the main SparkTask. However, there no effects from its execution, since the SELECT query + // output is piped to Shark's in-memory columnar storage builder, instead of a Hive tmp + // directory. + // TODO(harvey): Don't create a MoveTask in this case. val hiveFileSinkOp = terminalOps.head.localHiveOp mvTasks.foreach { moveTask => rootTasks.foreach { rootTask => From 6e52ca45c06834dbd7dfc2602940514a9e763d51 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 3 Nov 2013 18:19:32 -0800 Subject: [PATCH 206/331] 2nd round cleaning based on Harvey's feedback. --- .../shark/execution/LateralViewJoinOperator.scala | 3 +-- .../scala/shark/execution/MapSplitPruning.scala | 12 ++++++------ .../shark/execution/MemoryStoreSinkOperator.scala | 15 +++++++-------- .../scala/shark/execution/ScriptOperator.scala | 5 +++-- .../scala/shark/execution/UnionOperator.scala | 2 +- .../execution/optimization/ColumnPruner.scala | 4 +++- 6 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/main/scala/shark/execution/LateralViewJoinOperator.scala b/src/main/scala/shark/execution/LateralViewJoinOperator.scala index b6dedefa..b2d05953 100755 --- a/src/main/scala/shark/execution/LateralViewJoinOperator.scala +++ b/src/main/scala/shark/execution/LateralViewJoinOperator.scala @@ -25,8 +25,7 @@ import scala.reflect.BeanProperty import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluator, ExprNodeEvaluatorFactory} -import org.apache.hadoop.hive.ql.plan.SelectDesc -import org.apache.hadoop.hive.ql.plan.LateralViewJoinDesc +import org.apache.hadoop.hive.ql.plan.{LateralViewJoinDesc, SelectDesc} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index d0d2bd8e..d55f9fd8 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -17,16 +17,16 @@ package org.apache.hadoop.hive.ql.exec -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr import shark.memstore2.ColumnarStructObjectInspector.IDStructField import shark.memstore2.TablePartitionStats diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index f5e28129..f949f25c 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -161,14 +161,13 @@ class MemoryStoreSinkOperator extends TerminalOperator { val queryOutputRDD = outputRDD if (useUnionRDD) { // Handle an INSERT INTO command. - val previousRDDOpt: Option[RDD[TablePartition]] = - if (isHivePartitioned) { - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - databaseName, tableName).get - partitionedTable.getPartition(hivePartitionKey) - } else { - SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName).map(_.tableRDD) - } + val previousRDDOpt: Option[RDD[TablePartition]] = if (isHivePartitioned) { + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + databaseName, tableName).get + partitionedTable.getPartition(hivePartitionKey) + } else { + SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName).map(_.tableRDD) + } outputRDD = previousRDDOpt match { case Some(previousRDD) => { // If the RDD for a table or Hive-partition has already been created, then take a union diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 3de2a00e..2f9254f6 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -28,7 +28,8 @@ import scala.reflect.BeanProperty import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{ScriptOperator => HiveScriptOperator} -import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter, ScriptOperatorHelper} +import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter} +import org.apache.hadoop.hive.ql.exec.{ScriptOperatorHelper => HiveScriptOperatorHelper} import org.apache.hadoop.hive.ql.plan.ScriptDesc import org.apache.hadoop.hive.serde2.{Serializer, Deserializer} import org.apache.hadoop.io.{BytesWritable, Writable} @@ -174,7 +175,7 @@ class ScriptOperator extends UnaryOperator[ScriptDesc] { */ def getCommandAndEnvs(): (Seq[String], Map[String, String]) = { - val scriptOpHelper = new ScriptOperatorHelper(new HiveScriptOperator) + val scriptOpHelper = new HiveScriptOperatorHelper(new HiveScriptOperator) alias = scriptOpHelper.getAlias val cmdArgs = HiveScriptOperator.splitArgs(conf.getScriptCmd()) diff --git a/src/main/scala/shark/execution/UnionOperator.scala b/src/main/scala/shark/execution/UnionOperator.scala index 84470f65..e332739e 100755 --- a/src/main/scala/shark/execution/UnionOperator.scala +++ b/src/main/scala/shark/execution/UnionOperator.scala @@ -22,8 +22,8 @@ import java.util.{ArrayList, List => JavaList} import scala.collection.JavaConversions._ import scala.reflect.BeanProperty -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ReturnObjectInspectorResolver import org.apache.hadoop.hive.ql.plan.UnionDesc +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ReturnObjectInspectorResolver import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.StructField diff --git a/src/main/scala/shark/execution/optimization/ColumnPruner.scala b/src/main/scala/shark/execution/optimization/ColumnPruner.scala index d348fa3e..06ce0186 100644 --- a/src/main/scala/shark/execution/optimization/ColumnPruner.scala +++ b/src/main/scala/shark/execution/optimization/ColumnPruner.scala @@ -55,7 +55,9 @@ class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends * Computes the column names that are referenced in the Query */ private def computeColumnsToKeep( - op: Operator[_], cols: HashSet[String], parentOp: Operator[_] = null): Unit = { + op: Operator[_], + cols: HashSet[String], + parentOp: Operator[_] = null) { def nullGuard[T](s: JList[T]): Seq[T] = { if (s == null) Seq[T]() else s From 1d4318f302aaab89c72cfcaed3eeeb304eaa5199 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 3 Nov 2013 19:36:23 -0800 Subject: [PATCH 207/331] Oops ... rearrange import orders in ScriptOperator.scala. --- src/main/scala/shark/execution/ScriptOperator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/ScriptOperator.scala b/src/main/scala/shark/execution/ScriptOperator.scala index 2f9254f6..14203af4 100755 --- a/src/main/scala/shark/execution/ScriptOperator.scala +++ b/src/main/scala/shark/execution/ScriptOperator.scala @@ -27,8 +27,8 @@ import scala.reflect.BeanProperty import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.exec.{ScriptOperator => HiveScriptOperator} import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter} +import org.apache.hadoop.hive.ql.exec.{ScriptOperator => HiveScriptOperator} import org.apache.hadoop.hive.ql.exec.{ScriptOperatorHelper => HiveScriptOperatorHelper} import org.apache.hadoop.hive.ql.plan.ScriptDesc import org.apache.hadoop.hive.serde2.{Serializer, Deserializer} From 7b4db24973a8141295c35dfca61721352a3c5969 Mon Sep 17 00:00:00 2001 From: Xi Liu Date: Mon, 4 Nov 2013 14:09:51 -0800 Subject: [PATCH 208/331] fix byte buffer overflow caused by wrong size info --- src/main/scala/shark/memstore2/column/ColumnType.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/shark/memstore2/column/ColumnType.scala b/src/main/scala/shark/memstore2/column/ColumnType.scala index e839d901..7c432789 100644 --- a/src/main/scala/shark/memstore2/column/ColumnType.scala +++ b/src/main/scala/shark/memstore2/column/ColumnType.scala @@ -412,4 +412,6 @@ object GENERIC extends ColumnType[ByteStream.Output, ByteArrayRef](11, 16) { } override def newWritable() = new ByteArrayRef + + override def actualSize(v: ByteStream.Output): Int = v.getCount() + 4 } From 705389d7fc1633f5d6d16397da8fa88e7da67094 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 5 Nov 2013 11:51:41 -0800 Subject: [PATCH 209/331] FIXED SHARK-205: Map partition pruning for string column gets NPE on empty partitions. --- .../shark/execution/MapSplitPruning.scala | 13 ++-- .../shark/memstore2/column/ColumnStats.scala | 68 +++++++++++++------ .../shark/memstore2/ColumnStatsSuite.scala | 19 ++++-- 3 files changed, 66 insertions(+), 34 deletions(-) diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index d55f9fd8..584ad807 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -114,13 +114,14 @@ object MapSplitPruning { val invertValue: Boolean = invertEval.expr.getValue.asInstanceOf[Boolean] if (columnStats != null) { - val exists = (columnStats :>< (leftValue , rightValue)) - if (invertValue) !exists else exists - } else { - // If there is no stats on the column, don't prune. - true - } + val exists = (columnStats :>< (leftValue , rightValue)) + if (invertValue) !exists else exists + } else { + // If there is no stats on the column, don't prune. + true + } } + /** * Test whether we should keep the split as a candidate given the comparison * predicate. Return true if the split should be kept as a candidate, false if diff --git a/src/main/scala/shark/memstore2/column/ColumnStats.scala b/src/main/scala/shark/memstore2/column/ColumnStats.scala index 31270fa3..dce811d5 100644 --- a/src/main/scala/shark/memstore2/column/ColumnStats.scala +++ b/src/main/scala/shark/memstore2/column/ColumnStats.scala @@ -25,7 +25,8 @@ import org.apache.hadoop.io.Text /** - * Column level statistics, including range (min, max). + * Column level statistics, including range (min, max). We expect null values to be taken care + * of outside of the ColumnStats, so none of these stats should take null values. */ sealed trait ColumnStats[@specialized(Boolean, Byte, Short, Int, Long, Float, Double) T] extends Serializable { @@ -35,7 +36,6 @@ sealed trait ColumnStats[@specialized(Boolean, Byte, Short, Int, Long, Float, Do protected def _min: T protected def _max: T - def min: T = _min def max: T = _max @@ -67,27 +67,29 @@ object ColumnStats { class BooleanColumnStats extends ColumnStats[Boolean] { protected var _max = false protected var _min = true + override def append(v: Boolean) { if (v) _max = v else _min = v } + def :=(v: Any): Boolean = { v match { - case u:Boolean => _min <= u && _max >= u + case u: Boolean => _min <= u && _max >= u case _ => true } } def :>(v: Any): Boolean = { v match { - case u:Boolean => _max > u + case u: Boolean => _max > u case _ => true } } def :<(v: Any): Boolean = { v match { - case u:Boolean => _min < u + case u: Boolean => _min < u case _ => true } } @@ -97,6 +99,7 @@ object ColumnStats { class ByteColumnStats extends ColumnStats[Byte] { protected var _max = Byte.MinValue protected var _min = Byte.MaxValue + override def append(v: Byte) { if (v > _max) _max = v if (v < _min) _min = v @@ -104,21 +107,21 @@ object ColumnStats { def :=(v: Any): Boolean = { v match { - case u:Byte => _min <= u && _max >= u + case u: Byte => _min <= u && _max >= u case _ => true } } def :>(v: Any): Boolean = { v match { - case u:Byte => _max > u + case u: Byte => _max > u case _ => true } } def :<(v: Any): Boolean = { v match { - case u:Byte => _min < u + case u: Byte => _min < u case _ => true } } @@ -127,27 +130,29 @@ object ColumnStats { class ShortColumnStats extends ColumnStats[Short] { protected var _max = Short.MinValue protected var _min = Short.MaxValue + override def append(v: Short) { if (v > _max) _max = v if (v < _min) _min = v } + def :=(v: Any): Boolean = { v match { - case u:Short => _min <= u && _max >= u + case u: Short => _min <= u && _max >= u case _ => true } } def :>(v: Any): Boolean = { v match { - case u:Short => _max > u + case u: Short => _max > u case _ => true } } def :<(v: Any): Boolean = { v match { - case u:Short => _min < u + case u: Short => _min < u case _ => true } } @@ -184,14 +189,14 @@ object ColumnStats { def :>(v: Any): Boolean = { v match { - case u:Int => _max > u + case u: Int => _max > u case _ => true } } def :<(v: Any): Boolean = { v match { - case u:Int => _min < u + case u: Int => _min < u case _ => true } } @@ -228,27 +233,29 @@ object ColumnStats { class LongColumnStats extends ColumnStats[Long] { protected var _max = Long.MinValue protected var _min = Long.MaxValue + override def append(v: Long) { if (v > _max) _max = v if (v < _min) _min = v } + def :=(v: Any): Boolean = { v match { - case u:Long => _min <= u && _max >= u + case u: Long => _min <= u && _max >= u case _ => true } } def :>(v: Any): Boolean = { v match { - case u:Long => _max > u + case u: Long => _max > u case _ => true } } def :<(v: Any): Boolean = { v match { - case u:Long => _min < u + case u: Long => _min < u case _ => true } } @@ -257,20 +264,22 @@ object ColumnStats { class FloatColumnStats extends ColumnStats[Float] { protected var _max = Float.MinValue protected var _min = Float.MaxValue + override def append(v: Float) { if (v > _max) _max = v if (v < _min) _min = v } + def :=(v: Any): Boolean = { v match { - case u:Float => _min <= u && _max >= u + case u: Float => _min <= u && _max >= u case _ => true } } def :>(v: Any): Boolean = { v match { - case u:Float => _max > u + case u: Float => _max > u case _ => true } } @@ -286,10 +295,12 @@ object ColumnStats { class DoubleColumnStats extends ColumnStats[Double] { protected var _max = Double.MinValue protected var _min = Double.MaxValue + override def append(v: Double) { if (v > _max) _max = v if (v < _min) _min = v } + def :=(v: Any): Boolean = { v match { case u:Double => _min <= u && _max >= u @@ -315,10 +326,12 @@ object ColumnStats { class TimestampColumnStats extends ColumnStats[Timestamp] { protected var _max = new Timestamp(0) protected var _min = new Timestamp(Long.MaxValue) + override def append(v: Timestamp) { if (v.compareTo(_max) > 0) _max = v if (v.compareTo(_min) < 0) _min = v } + def :=(v: Any): Boolean = { v match { case u: Timestamp => _min.compareTo(u) <=0 && _max.compareTo(u) >= 0 @@ -345,8 +358,12 @@ object ColumnStats { // Note: this is not Java serializable because Text is not Java serializable. protected var _max: Text = null protected var _min: Text = null - + def :=(v: Any): Boolean = { + if (_max eq null) { + // This partition doesn't contain any non-null strings in this column. Return false. + return false + } v match { case u: Text => _min.compareTo(u) <= 0 && _max.compareTo(u) >= 0 case u: String => this := new Text(u) @@ -355,6 +372,10 @@ object ColumnStats { } def :>(v: Any): Boolean = { + if (_max eq null) { + // This partition doesn't contain any non-null strings in this column. Return false. + return false + } v match { case u: Text => _max.compareTo(u) > 0 case u: String => this :> new Text(u) @@ -363,14 +384,19 @@ object ColumnStats { } def :<(v: Any): Boolean = { + if (_max eq null) { + // This partition doesn't contain any non-null strings in this column. Return false. + return false + } v match { - case u:Text => _min.compareTo(u) < 0 + case u: Text => _min.compareTo(u) < 0 case u: String => this :< new Text(u) case _ => true } } override def append(v: Text) { + assert(!(v eq null)) // Need to make a copy of Text since Text is not immutable and we reuse // the same Text object in serializer to mitigate frequent GC. if (_max == null) { @@ -382,7 +408,7 @@ object ColumnStats { _min = new Text(v) } else if (v.compareTo(_min) < 0) { _min.set(v.getBytes(), 0, v.getLength()) - } + } } override def readExternal(in: ObjectInput) { diff --git a/src/test/scala/shark/memstore2/ColumnStatsSuite.scala b/src/test/scala/shark/memstore2/ColumnStatsSuite.scala index 21e55b21..31968103 100644 --- a/src/test/scala/shark/memstore2/ColumnStatsSuite.scala +++ b/src/test/scala/shark/memstore2/ColumnStatsSuite.scala @@ -54,7 +54,7 @@ class ColumnStatsSuite extends FunSuite { } test("ByteColumnStats") { - var c = new ColumnStats.ByteColumnStats + val c = new ColumnStats.ByteColumnStats c.append(0) assert(c.min == 0 && c.max == 0) c.append(1) @@ -72,7 +72,7 @@ class ColumnStatsSuite extends FunSuite { } test("ShortColumnStats") { - var c = new ColumnStats.ShortColumnStats + val c = new ColumnStats.ShortColumnStats c.append(0) assert(c.min == 0 && c.max == 0) c.append(1) @@ -123,7 +123,7 @@ class ColumnStatsSuite extends FunSuite { } test("LongColumnStats") { - var c = new ColumnStats.LongColumnStats + val c = new ColumnStats.LongColumnStats c.append(0) assert(c.min == 0 && c.max == 0) c.append(1) @@ -140,7 +140,7 @@ class ColumnStatsSuite extends FunSuite { } test("FloatColumnStats") { - var c = new ColumnStats.FloatColumnStats + val c = new ColumnStats.FloatColumnStats c.append(0) assert(c.min == 0 && c.max == 0) c.append(1) @@ -157,7 +157,7 @@ class ColumnStatsSuite extends FunSuite { } test("DoubleColumnStats") { - var c = new ColumnStats.DoubleColumnStats + val c = new ColumnStats.DoubleColumnStats c.append(0) assert(c.min == 0 && c.max == 0) c.append(1) @@ -174,7 +174,7 @@ class ColumnStatsSuite extends FunSuite { } test("TimestampColumnStats") { - var c = new ColumnStats.TimestampColumnStats + val c = new ColumnStats.TimestampColumnStats val ts1 = new Timestamp(1000) val ts2 = new Timestamp(2000) val ts3 = new Timestamp(1500) @@ -197,8 +197,13 @@ class ColumnStatsSuite extends FunSuite { test("StringColumnStats") { implicit def T(str: String): Text = new Text(str) - var c = new ColumnStats.StringColumnStats + val c = new ColumnStats.StringColumnStats assert(c.min == null && c.max == null) + + assert(!(c :> "test")) + assert(!(c :< "test")) + assert(!(c == "test")) + c.append("a") assert(c.min.equals(T("a")) && c.max.equals(T("a"))) From f3774fa9bea720b66ed5c0562b1b3efe3b7db9e1 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 5 Nov 2013 21:08:18 -0800 Subject: [PATCH 210/331] Add a helper function in HiveUtils that creates a UnionOI for a schema that includes partitioned columns. --- src/main/scala/shark/util/HiveUtils.scala | 31 ++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 28939f92..7c4e1191 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -17,14 +17,22 @@ package shark.util -import java.util.{ArrayList => JavaArrayList, HashMap => JavaHashMap, HashSet => JavaHashSet} +import java.util.{ArrayList => JavaArrayList, Arrays => JavaArrays} +import java.util.{HashMap => JavaHashMap, HashSet => JavaHashSet, Map => JavaMap} +import java.util.Properties import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.objectinspector.UnionStructObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} import org.apache.hadoop.hive.ql.plan.AlterTableDesc @@ -34,6 +42,7 @@ import shark.SharkContext import shark.api.{DataType, DataTypes} + private[shark] object HiveUtils { def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = { @@ -52,6 +61,26 @@ private[shark] object HiveUtils { case DataTypes.STRING => PrimitiveObjectInspectorFactory.javaStringObjectInspector } + def makeUnionOIForPartitionedTable( + partProps: Properties, + partSerDe: Deserializer): UnionStructObjectInspector = { + val partCols = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + val partColNames = new JavaArrayList[String] + val partColObjectInspectors = new JavaArrayList[ObjectInspector] + partCols.trim().split("/").foreach { colName => + partColNames.add(colName) + partColObjectInspectors.add(PrimitiveObjectInspectorFactory.javaStringObjectInspector) + } + + val partColObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( + partColNames, partColObjectInspectors) + val oiList = JavaArrays.asList( + partSerDe.getObjectInspector().asInstanceOf[StructObjectInspector], + partColObjectInspector.asInstanceOf[StructObjectInspector]) + // New oi is union of table + partition object inspectors + ObjectInspectorFactory.getUnionStructObjectInspector(oiList) + } + /** * Execute the create table DDL operation against Hive's metastore. */ From 7f39c1ea6d02f24a8f0190f5c1f377e5a8ce3974 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 5 Nov 2013 21:10:33 -0800 Subject: [PATCH 211/331] Hive compliance: pass a NULL partition specification in HiveUtils#alterSerDeInHive() helper. This indicates a non-partitioned table, and makes it slightly easier to deal with NULLs/Options in the code. --- src/main/scala/shark/util/HiveUtils.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 7c4e1191..2c38bdc5 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -132,16 +132,21 @@ private[shark] object HiveUtils { * partition) entry in the Hive metastore. Returns `true` if successful. * * @tableName Name of table being altered. - * @partitionSpec Map of (partition col, partition key) pairs for which the SerDe is being - * altered. NULL if the table isn't Hive-partitioned. + * @partitionSpecOpt Map of (partition col, partition key) pairs for which the SerDe is being + * altered. `None` if the table isn't Hive-partitioned. * @serDeName Class name of new SerDe to use. * @hiveConf Configuration associated with the current SessionState. */ def alterSerdeInHive( tableName: String, - partitionSpec: JavaHashMap[String, String], + partitionSpecOpt: Option[JavaMap[String, String]], serDeName: String, hiveConf: HiveConf = new HiveConf): Boolean = { + val partitionSpec = if (partitionSpecOpt.isDefined) { + partitionSpecOpt.get.asInstanceOf[JavaHashMap[String, String]] + } else { + null + } val alterTableDesc = new AlterTableDesc(AlterTableDesc.AlterTableTypes.ADDSERDE) alterTableDesc.setOldName(tableName) alterTableDesc.setSerdeName(serDeName) From 6c3523d781a983facf10b13835488f2d1af813c6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 5 Nov 2013 21:12:11 -0800 Subject: [PATCH 212/331] New method in Utils that creates a PathFilter based on the snapshot of a given path. Useful in computing deltas (i.e., new files/directories) for HadoopRDD loads. --- src/main/scala/shark/Utils.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/Utils.scala b/src/main/scala/shark/Utils.scala index 66a42f5c..7bf3a75f 100644 --- a/src/main/scala/shark/Utils.scala +++ b/src/main/scala/shark/Utils.scala @@ -19,8 +19,8 @@ package shark import java.io.BufferedReader import java.util.{Map => JMap} - import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{Path, PathFilter} object Utils { @@ -94,4 +94,17 @@ object Utils { new BufferedReader(new InputStreamReader(s3obj.getDataInputStream())) } + /** + * Returns a filter that accepts files missing from the current snapshot of the `path` directory. + */ + def createSnapshotFilter(path: Path, conf: Configuration): PathFilter = { + val fs = path.getFileSystem(conf) + val currentFiles = fs.listStatus(path).map(_.getPath).toSet + val fileFilter = new PathFilter() { + override def accept(path: Path) = { + (!path.getName().startsWith(".") && !currentFiles.contains(path)) + } + } + fileFilter + } } From 043357bfc58a46c3aeaaa25d42081f7c7cdba8ac Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 5 Nov 2013 21:14:18 -0800 Subject: [PATCH 213/331] Add `unifyView` fields in Shark Tables. --- src/main/scala/shark/memstore2/MemoryTable.scala | 6 +++--- src/main/scala/shark/memstore2/PartitionedMemoryTable.scala | 5 +++-- src/main/scala/shark/memstore2/Table.scala | 2 ++ 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 020a112f..25577a32 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -30,8 +30,9 @@ private[shark] class MemoryTable( tableName: String, cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel) - extends Table(tableName, cacheMode, preferredStorageLevel) { + preferredStorageLevel: StorageLevel, + unifiedView: Boolean) + extends Table(tableName, cacheMode, preferredStorageLevel, unifiedView) { // RDD that contains the contents of this table. private var _tableRDD: RDD[TablePartition] = _ @@ -39,5 +40,4 @@ class MemoryTable( def tableRDD: RDD[TablePartition] = _tableRDD def tableRDD_= (rdd: RDD[TablePartition]) = _tableRDD = rdd - } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index fb63596d..a14002c8 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -39,8 +39,9 @@ private[shark] class PartitionedMemoryTable( tableName: String, cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel) - extends Table(tableName, cacheMode, preferredStorageLevel) { + preferredStorageLevel: StorageLevel, + unifiedView: Boolean) + extends Table(tableName, cacheMode, preferredStorageLevel, unifiedView) { /** * A simple, mutable wrapper for an RDD. This is needed so that a entry maintained by a diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 5e9af782..20c31fd1 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -35,3 +35,5 @@ private[shark] abstract class Table( var tableName: String, var cacheMode: CacheType.CacheType, var preferredStorageLevel: StorageLevel) + var preferredStorageLevel: StorageLevel, + var unifyView: Boolean) { From d6e52719426f9c392d090d744b3bb853ba2ac5ab Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 5 Nov 2013 21:19:19 -0800 Subject: [PATCH 214/331] Enable deserializer specification in HadoopTableReader. The default would be the deserializer specified by the table descriptor. One case for which a different deserializer is specified is for updates to a unified view. The table descriptor's deserializer is a ColumnarSerDe, so a deserialize()-capable disk/Hadoop deserializer is passed directly to makeRDDForTable() or makeRDDForPartitionedTable(). --- .../shark/execution/HadoopTableReader.scala | 35 +++++++++++++------ 1 file changed, 24 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index 87d7a6ee..78dfa1e0 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -25,8 +25,9 @@ import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUM import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} -import org.apache.hadoop.io.Writable +import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.io.Writable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.SerializableWritable @@ -53,14 +54,22 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf def broadcastedHiveConf = _broadcastedHiveConf + def hiveConf = _broadcastedHiveConf.value.value + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = - makeRDDForTable(hiveTable, filterOpt = None) - + makeRDDForTable( + hiveTable, + None /* fitlerOpt */, + _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]]) + /** * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed * RDD that contains deserialized rows. */ - def makeRDDForTable(hiveTable: HiveTable, filterOpt: Option[PathFilter] = None): RDD[_] = { + def makeRDDForTable( + hiveTable: HiveTable, + filterOpt: Option[PathFilter], + deserializerClass: Class[_ <: Deserializer]): RDD[_] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") @@ -79,7 +88,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val deserializer = tableDesc.getDeserializerClass().newInstance() + val deserializer = deserializerClass.newInstance().asInstanceOf[Deserializer] deserializer.initialize(hconf, tableDesc.getProperties) // Deserialize each Writable to get the row value. @@ -93,8 +102,12 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf deserializedHadoopRDD } - override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = - makeRDDForPartitionedTable(partitions, filterOpt = None) + override def makeRDDForPartitionedTable( + partitions: Seq[HivePartition]): RDD[_] = { + val partitionToDeserializer = partitions.map(part => + (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap + makeRDDForPartitionedTable(partitionToDeserializer, None /* filterOpt */) + } /** * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive @@ -102,9 +115,9 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf * 'PARTITION BY'. */ def makeRDDForPartitionedTable( - partitions: Seq[HivePartition], + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[_] = { - val hivePartitionRDDs = partitions.map { partition => + val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) val partPath = partition.getPartitionPath val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) @@ -113,7 +126,6 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf // Get partition field info val partSpec = partDesc.getPartSpec() val partProps = partDesc.getProperties() - val partDeserializer = partDesc.getDeserializerClass() val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) // Partitioning columns are delimited by "/" @@ -143,7 +155,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf rowWithPartArr.asInstanceOf[Object] } } - } + }.toSeq // Even if we don't use any partitions, we still need an empty RDD if (hivePartitionRDDs.size == 0) { new EmptyRDD[Object](SharkEnv.sc) @@ -190,6 +202,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) } + } object HadoopTableReader { From be16618728e13e8653743c065e814f67fb65f865 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 5 Nov 2013 21:25:49 -0800 Subject: [PATCH 215/331] Add fields to Table and PartitionedMemoryTable to track SerDes, for deserialize()-ing disk reads. --- .../shark/memstore2/PartitionedMemoryTable.scala | 13 +++++++++++-- src/main/scala/shark/memstore2/Table.scala | 9 ++++++++- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index a14002c8..49e82284 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -58,6 +58,9 @@ class PartitionedMemoryTable( private var _keyToPartitions: ConcurrentMap[String, RDDValue] = new ConcurrentJavaHashMap[String, RDDValue]() + private var _keyToDiskSerDes: ConcurrentMap[String, String] = + new ConcurrentJavaHashMap[String, String]() + // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: // `TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")`. @@ -122,11 +125,17 @@ class PartitionedMemoryTable( _cachePolicy = newPolicy } + def setDiskSerDe(partitionKey: String, serDe: String) = _keyToDiskSerDes.put(partitionKey, serDe) + + def getDiskSerDe(partitionKey: String): Option[String] = _keyToDiskSerDes.get(partitionKey) + def cachePolicy: CachePolicy[String, RDDValue] = _cachePolicy - /** Returns an immutable view of the String->RDD mapping to external callers */ + /** Returns an immutable view of (partition key -> RDD) mappings to external callers */ def keyToPartitions: collection.immutable.Map[String, RDD[TablePartition]] = { - return _keyToPartitions.mapValues(_.rdd).toMap + _keyToPartitions.mapValues(_.rdd).toMap } + /** Returns an immutable view of (partition key -> SerDe name) mappings to external callers */ + def keyToDiskSerDes: collection.immutable.Map[String, String] = _keyToDiskSerDes.toMap } diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 20c31fd1..63fe9ece 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -34,6 +34,13 @@ import org.apache.spark.storage.StorageLevel private[shark] abstract class Table( var tableName: String, var cacheMode: CacheType.CacheType, - var preferredStorageLevel: StorageLevel) var preferredStorageLevel: StorageLevel, var unifyView: Boolean) { + + // SerDe used to read from and write to disk. + private var _diskSerDe: String = _ + + def diskSerDe: String = _diskSerDe + + def diskSerDe_= (newSerDe: String) = _diskSerDe = newSerDe +} From ceab5f7d7b41cf67710ea7bbfc434ffbf08b49a4 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 6 Nov 2013 04:55:23 -0800 Subject: [PATCH 216/331] Account for unified views in Shark table creation in MemoryMetadataManager. --- .../shark/memstore2/MemoryMetadataManager.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index ca64fd27..950df71e 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -71,10 +71,11 @@ class MemoryMetadataManager { databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel + preferredStorageLevel: StorageLevel, + unifyView: Boolean ): MemoryTable = { val tableKey = makeTableKey(databaseName, tableName) - var newTable = new MemoryTable(tableKey, cacheMode, preferredStorageLevel) + var newTable = new MemoryTable(tableKey, cacheMode, preferredStorageLevel, unifyView) _keyToTable.put(tableKey, newTable) return newTable } @@ -84,10 +85,15 @@ class MemoryMetadataManager { tableName: String, cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, + unifyView: Boolean, tblProps: JavaMap[String, String] ): PartitionedMemoryTable = { val tableKey = makeTableKey(databaseName, tableName) - var newTable = new PartitionedMemoryTable(tableKey, cacheMode, preferredStorageLevel) + var newTable = new PartitionedMemoryTable( + tableKey, + cacheMode, + preferredStorageLevel, + unifyView) // Determine the cache policy to use and read any user-specified cache settings. val cachePolicyStr = tblProps.getOrElse(SharkConfVars.CACHE_POLICY.varname, SharkConfVars.CACHE_POLICY.defaultVal) From 5c1b5aa4cbb0534f5030e55437e5c8491616c1f6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 5 Nov 2013 21:22:18 -0800 Subject: [PATCH 217/331] Additions to SparkLoadTask and SparkLoadSemanticAnalyzer to support CTAS, INSERT INTO, INSERT OVERWRITE, LOAD INTO and LOAD OVERWRITE. --- .../scala/shark/execution/SparkLoadTask.scala | 235 ++++++++++++++---- .../parse/SharkLoadSemanticAnalyzer.scala | 34 +-- 2 files changed, 200 insertions(+), 69 deletions(-) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 15d57b3b..1de575fd 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -18,19 +18,20 @@ package shark.execution -import java.util.{Properties, Map => JavaMap} +import java.util.{HashMap => JavaHashMap, Properties, Map => JavaMap} import scala.collection.JavaConversions._ - import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.{Context, DriverContext} import org.apache.hadoop.hive.ql.exec.{Task => HiveTask, Utilities} -import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.plan.api.StageType +import org.apache.hadoop.hive.serde.Constants; +import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat} @@ -38,110 +39,212 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.SerializableWritable +import org.apache.spark.storage.StorageLevel import shark.{LogHelper, SharkEnv} import shark.execution.serialization.KryoSerializer import shark.memstore2._ +import shark.util.HiveUtils private[shark] class SparkLoadWork( - val hiveTable: HiveTable, - val partSpecsOpt: Option[JavaMap[String, String]], - val isOverwrite: Boolean, + val databaseName: String, + val tableName: String, + val partSpecOpt: Option[JavaMap[String, String]], + val commandType: SparkLoadWork.CommandTypes.Type, + val storageLevel: StorageLevel, val pathFilter: Option[PathFilter]) extends java.io.Serializable +object SparkLoadWork { + object CommandTypes extends Enumeration { + type Type = Value + val OVERWRITE, INSERT, NEW_ENTRY = Value + } +} + private[shark] class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHelper { override def execute(driveContext: DriverContext): Int = { logDebug("Executing " + this.getClass.getName) - val hiveTable = work.hiveTable + val databaseName = work.databaseName + val tableName = work.tableName + val hiveTable = Hive.get(conf).getTable(databaseName, tableName) val tableDesc = Utilities.getTableDesc(hiveTable) - val tableNameSplit = tableDesc.getTableName().split('.') - val databaseName = tableNameSplit(0) - val tableName = tableNameSplit(1) val oi = hiveTable.getDeserializer().getObjectInspector().asInstanceOf[StructObjectInspector] + val sharkTableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) val hadoopReader = new HadoopTableReader(tableDesc, conf) - val inputRDD = if (hiveTable.isPartitioned) { - val partition = db.getPartition(hiveTable, work.partSpecsOpt.get, false /* forceCreate */) - hadoopReader.makeRDDForPartitionedTable(Seq(partition), work.pathFilter) - } else { - hadoopReader.makeRDDForTable(hiveTable, work.pathFilter) - } - val (tablePartitionRDD, tableStats) = transformInputRdd( - inputRDD, - tableDesc.getProperties, - hadoopReader.broadcastedHiveConf, - hiveTable.getDeserializer.getObjectInspector().asInstanceOf[StructObjectInspector]) - - // TODO(harvey): This part overlaps with, though is slightly cleaner than, execution code in - // MemoryStoreSinkOperator. - // LOAD DATA INPATH behaves like an INSERT, so use a UnionRDD if no OVERWRITE is specified. - SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { - case Some(table) => { + // TODO(harvey): Cleanup. This part overlaps with execution code in MemoryStoreSinkOperator. + sharkTableOpt match { + case Some(sharkTable) => { var unionWithPreviousStats = false - table match { + sharkTable match { case memoryTable: MemoryTable => { - memoryTable.tableRDD = if (work.isOverwrite) { - tablePartitionRDD - } else { - unionWithPreviousStats = true - RDDUtils.unionAndFlatten(memoryTable.tableRDD, tablePartitionRDD) + val serDe = Class.forName( + memoryTable.diskSerDe).newInstance.asInstanceOf[Deserializer] + serDe.initialize(conf, tableDesc.getProperties) + val inputRDD = hadoopReader.makeRDDForTable( + hiveTable, + work.pathFilter, + serDe.getClass) + val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + inputRDD, + tableDesc.getProperties, + work.storageLevel, + hadoopReader.broadcastedHiveConf, + serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) + memoryTable.tableRDD = work.commandType match { + case SparkLoadWork.CommandTypes.OVERWRITE => tablePartitionRDD + case SparkLoadWork.CommandTypes.INSERT => { + RDDUtils.unionAndFlatten(tablePartitionRDD, memoryTable.tableRDD) + } + case SparkLoadWork.CommandTypes.NEW_ENTRY => { + throw new Exception( + "Invalid state: table %s already exists in memory".format(tableName)) + } } + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } case partitionedTable: PartitionedMemoryTable => { val partCols = hiveTable.getPartCols.map(_.getName) - val partSpecs = work.partSpecsOpt.get + val partSpecs = work.partSpecOpt.get val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpecs) + val partition = db.getPartition(hiveTable, partSpecs, false /* forceCreate */) + val partSerDe = Class.forName(partitionedTable.getDiskSerDe(partitionKey).getOrElse( + partitionedTable.diskSerDe)).newInstance.asInstanceOf[Deserializer] + val partSchema = partition.getSchema + partSerDe.initialize(conf, partSchema) + val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) + val inputRDD = hadoopReader.makeRDDForPartitionedTable( + Map(partition -> partSerDe.getClass), work.pathFilter) + val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + inputRDD, + addPartitionInfoToSerDeProps(partCols, new Properties(partition.getSchema)), + work.storageLevel, + hadoopReader.broadcastedHiveConf, + unionOI) partitionedTable.getPartition(partitionKey) match { case Some(previousRDD) => { - partitionedTable.updatePartition( - partitionKey, RDDUtils.unionAndFlatten(previousRDD, tablePartitionRDD)) - unionWithPreviousStats = true + work.commandType match { + case SparkLoadWork.CommandTypes.OVERWRITE => { + partitionedTable.putPartition(partitionKey, tablePartitionRDD) + } + case SparkLoadWork.CommandTypes.INSERT => { + partitionedTable.updatePartition( + partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) + // Union stats for the previous RDD with the new RDD loaded. + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { + case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) + case _ => Unit + } + } + case SparkLoadWork.CommandTypes.NEW_ENTRY => { + throw new Exception( + "Invalid state: table %s already exists in memory".format(tableName)) + } + } } case None => { - // Either `isOverwrite` is true, or the partition being updated does not currently - // exist. + // The partition being updated does not currently exist. Create and set a new + // partition key entry, and register a shutdown callback in the Shark metastore. partitionedTable.putPartition(partitionKey, tablePartitionRDD) } } + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } } - val newStats = if (unionWithPreviousStats) { - // Union the stats maps. - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { - case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) - case _ => tableStats - } - } else { - tableStats - } - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, newStats.toMap) } case None => { - throw new Exception("Couldn't find the target table for a LOAD.") + // Couldn't find the target table for a LOAD. + assert(work.commandType == SparkLoadWork.CommandTypes.NEW_ENTRY) + + // Add the table to-be-cached to the Shark metastore. + val tblProps = hiveTable.getParameters() + val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + val inputRDD = if (hiveTable.isPartitioned) { + val partition = db.getPartition(hiveTable, work.partSpecOpt.get, false /* forceCreate */) + hadoopReader.makeRDDForPartitionedTable(Seq(partition)) + } else { + hadoopReader.makeRDDForTable(hiveTable) + } + work.partSpecOpt match { + case Some(partSpecs) => { + val hivePartition = Hive.get(conf).getPartition( + hiveTable, + partSpecs, + false /* forceCreate */) + val partSerDe = hivePartition.getDeserializer + val partCols = hiveTable.getPartCols.map(_.getName) + val partSchema = hivePartition.getSchema + partSerDe.initialize(conf, partSchema) + val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) + // Partitioned table. + val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + inputRDD, + addPartitionInfoToSerDeProps(partCols, new Properties(hivePartition.getSchema)), + work.storageLevel, + hadoopReader.broadcastedHiveConf, + unionOI) + val newTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( + databaseName, + tableName, + cacheMode, + preferredStorageLevel, + unifyView = true, + tblProps) + val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr( + hiveTable.getPartCols.map(_.getName), partSpecs) + newTable.putPartition(partitionKey, tablePartitionRDD) + newTable.setDiskSerDe(partitionKey, hivePartition.getDeserializer.getClass.getName) + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) + } + case None => { + val serDe = hiveTable.getDeserializer + val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + inputRDD, + tableDesc.getProperties, + work.storageLevel, + hadoopReader.broadcastedHiveConf, + serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) + // Non-partitioned table. + val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( + databaseName, tableName, cacheMode, preferredStorageLevel, unifyView = true) + newTable.tableRDD = tablePartitionRDD + // Record what the previous SerDe was. + newTable.diskSerDe = serDe.getClass.getName + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) + } + } + // Set the new SerDe to be a ColumnarSerDe, since the data has been cached. + HiveUtils.alterSerdeInHive( + tableName, + work.partSpecOpt, + classOf[ColumnarSerDe].getName, + conf) } } - // Success! 0 } - def transformInputRdd( + def transformAndMaterializeInput( inputRdd: RDD[_], - tableProps: Properties, + serDeProps: Properties, + storageLevel: StorageLevel, broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], oi: StructObjectInspector) = { val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) val serializedOI = KryoSerializer.serialize(oi) val transformedRdd = inputRdd.mapPartitionsWithIndex { case (partIndex, partIter) => val serde = new ColumnarSerDe - serde.initialize(broadcastedHiveConf.value.value, tableProps) + serde.initialize(broadcastedHiveConf.value.value, serDeProps) val oi = KryoSerializer.deserialize[ObjectInspector](serializedOI) var builder: Writable = null partIter.foreach { row => @@ -156,6 +259,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe Iterator(builder.asInstanceOf[TablePartitionBuilder].build) } } + transformedRdd.persist(storageLevel) transformedRdd.context.runJob( transformedRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) (transformedRdd, statsAcc.value) @@ -172,6 +276,29 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe targetStatsMap } + def addPartitionInfoToSerDeProps( + partCols: Seq[String], + serDeProps: Properties): Properties = { + // Delimited by "," + var columnNameProperty: String = serDeProps.getProperty(Constants.LIST_COLUMNS) + // NULL if column types are missing. By default, the SerDeParameters initialized by the + // ColumnarSerDe will treat all columns as having string types. + // Delimited by ":" + var columnTypeProperty: String = serDeProps.getProperty(Constants.LIST_COLUMN_TYPES) + + for (partColName <- partCols) { + columnNameProperty += "," + partColName + } + if (columnTypeProperty != null) { + for (partColName <- partCols) { + columnTypeProperty += ":" + Constants.STRING_TYPE_NAME + } + } + serDeProps.setProperty(Constants.LIST_COLUMNS, columnNameProperty) + serDeProps.setProperty(Constants.LIST_COLUMN_TYPES, columnTypeProperty) + serDeProps + } + override def getType = StageType.MAPRED override def getName = "MAPRED-LOAD-SPARK" diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index c5d8d16d..89002734 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -18,18 +18,17 @@ package shark.parse import scala.collection.JavaConversions._ - import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{CopyTask, MoveTask, TaskFactory} import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, LoadSemanticAnalyzer} import org.apache.hadoop.hive.ql.plan._ - import shark.execution.SparkLoadWork import shark.{LogHelper, SharkEnv} +import shark.Utils -class SharkLoadSemanticAnalyzer(hiveConf: HiveConf) extends LoadSemanticAnalyzer(hiveConf) { +class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(conf) { override def analyzeInternal(ast: ASTNode): Unit = { // Delegate to the LoadSemanticAnalyzer parent for error checking the source path formatting. @@ -43,38 +42,43 @@ class SharkLoadSemanticAnalyzer(hiveConf: HiveConf) extends LoadSemanticAnalyzer val tableName = getTableName(tableASTNode) val databaseName = db.getCurrentDatabase() - if (SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { + val tableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) + if (tableOpt.exists(_.unifyView)) { // Find the arguments needed to instantiate a SparkLoadWork. val tableSpec = new BaseSemanticAnalyzer.tableSpec(db, conf, tableASTNode) + val preferredStorageLevel = tableOpt.get.preferredStorageLevel val hiveTable = tableSpec.tableHandle val partSpecOpt = Option(tableSpec.getPartSpec()) - var partition: Partition = null val dataPath = if (partSpecOpt.isEmpty) { // Non-partitioned table. hiveTable.getPath } else { // Partitioned table. - partition = db.getPartition(hiveTable, partSpecOpt.get, false /* forceCreate */) + val partition = db.getPartition(hiveTable, partSpecOpt.get, false /* forceCreate */) partition.getPartitionPath } val moveTask = getMoveTask() - val isOverwrite = moveTask.getWork.getLoadTableWork.getReplace() + val loadCommandType = if (moveTask.getWork.getLoadTableWork.getReplace()) { + SparkLoadWork.CommandTypes.OVERWRITE + } else { + SparkLoadWork.CommandTypes.INSERT + } // Capture a snapshot of the data directory being read. When executed, SparkLoadTask will // determine the input paths to read using a filter that only accepts files not included in // snapshot set (i.e., the accepted file is a new one created by the Hive load process). - val fs = dataPath.getFileSystem(hiveConf) - val currentFiles = fs.listStatus(dataPath).map(_.getPath).toSet - val fileFilter = new PathFilter() { - override def accept(path: Path) = { - (!path.getName().startsWith(".") && !currentFiles.contains(path)) - } - } + val fileFilter = Utils.createSnapshotFilter(dataPath, conf) // Create a SparkLoadTask that will use a HadoopRDD to read from the source directory. Set it // to be a dependent task of the LoadTask so that the SparkLoadTask is executed only if the // Hive task executes successfully. - val sparkLoadWork = new SparkLoadWork(hiveTable, partSpecOpt, isOverwrite, Some(fileFilter)) + val sparkLoadWork = new SparkLoadWork( + databaseName, + tableName, + partSpecOpt, + loadCommandType, + preferredStorageLevel, + Some(fileFilter)) moveTask.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } } From 2931e91eff46fcd90cfaf254a1f4bbcf7868fbeb Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 6 Nov 2013 00:21:32 -0800 Subject: [PATCH 218/331] Initial implementation for semantic analysis of unified views. --- src/main/scala/shark/SharkConfVars.scala | 3 + .../shark/execution/OperatorFactory.scala | 13 ++ src/main/scala/shark/parse/QueryBlock.scala | 37 +++- .../shark/parse/SharkSemanticAnalyzer.scala | 201 +++++++++++++----- 4 files changed, 199 insertions(+), 55 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 70af8aa0..519310fa 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -52,6 +52,9 @@ object SharkConfVars { // "shark.cache.partition.cachePolicy" property above. val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.policy.maxSize", "10") + // Default value for the "shark.cache.unify" table property. + val DEFAULT_UNIFY_FLAG = new ConfVar("shark.cache.unify.default", true) + // If true, then cache any table whose name ends in "_cached". val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 66b4d592..26ec7742 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator} import org.apache.hadoop.hive.ql.exec.GroupByPreShuffleOperator import org.apache.hadoop.hive.ql.exec.{Operator => HOperator} import org.apache.hadoop.hive.ql.metadata.HiveException +import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.storage.StorageLevel @@ -78,6 +79,18 @@ object OperatorFactory extends LogHelper { _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } + def createUnifiedViewFileOutputPlan( + hiveTerminalOp: HOperator[_<:HiveDesc], + diskSerDe: String): TerminalOperator = { + var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] + val terminalOp = createSharkFileOutputPlan(hiveTerminalOp) + val fileSinkDesc = hiveOp.getConf + val tableDesc = fileSinkDesc.getTableInfo() + val serDe = Class.forName(diskSerDe).asInstanceOf[Class[Deserializer]] + tableDesc.setDeserializerClass(serDe) + terminalOp + } + def createSharkRddOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 16647605..04c94693 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -18,6 +18,10 @@ package shark.parse import org.apache.hadoop.hive.ql.parse.{QB => HiveQueryBlock} +import org.apache.hadoop.hive.ql.plan.CreateTableDesc +import org.apache.hadoop.hive.ql.plan.TableDesc + +import org.apache.spark.storage.StorageLevel import shark.memstore2.CacheType import shark.memstore2.CacheType._ @@ -31,11 +35,34 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) extends HiveQueryBlock(outerID, alias, isSubQuery) { // The CacheType for the table that will be created from CREATE TABLE/CTAS. - private var cacheModeForCreateTable = CacheType.NONE + private var _cacheModeForCreateTable = CacheType.NONE + + private var _preferredStorageLevel: StorageLevel = StorageLevel.NONE + + // Whether the created to be created or the table specified by CACHED should be backed by disk. + private var _unifyView = false + + // TableDesc for a table being updated by an INSERT. + private var _targetTableDesc: TableDesc = _ + + def cacheModeForCreateTable_= (mode: CacheType) = _cacheModeForCreateTable = mode + + def cacheModeForCreateTable: CacheType = _cacheModeForCreateTable + + def preferredStorageLevel_= (storageLevel: StorageLevel) = _preferredStorageLevel = storageLevel + + def preferredStorageLevel: StorageLevel = _preferredStorageLevel + + def unifyView_= (shouldUnify: Boolean) = _unifyView = shouldUnify + + def unifyView: Boolean = _unifyView + + def targetTableDesc: TableDesc = _targetTableDesc + + def targetTableDesc_= (desc: TableDesc) = _targetTableDesc = desc - def setCacheModeForCreateTable(cacheModeToSet: CacheType) { - cacheModeForCreateTable = cacheModeToSet - } + // Hive uses "tableDesc" to refer to the CreateTableDesc... + def createTableDesc: CreateTableDesc = super.getTableDesc - def getCacheModeForCreateTable(): CacheType = cacheModeForCreateTable + def createTableDesc_= (desc: CreateTableDesc) = super.setTableDesc(desc) } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index edc51700..8f73126c 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -17,10 +17,11 @@ package shark.parse -import java.lang.reflect.Method import java.util.ArrayList +import java.util.{HashSet => JavaHashSet} import java.util.{List => JavaList} import java.util.{Map => JavaMap} +import java.lang.reflect.Method import scala.collection.JavaConversions._ @@ -28,9 +29,11 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} import org.apache.hadoop.hive.metastore.Warehouse -import org.apache.hadoop.hive.ql.exec.{DDLTask, FetchTask, MoveTask, TaskFactory} -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator, - Operator => HiveOperator} +import org.apache.hadoop.hive.ql.exec.{DDLTask, FetchTask} +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} +import org.apache.hadoop.hive.ql.exec.MoveTask +import org.apache.hadoop.hive.ql.exec.{Operator => HiveOperator} +import org.apache.hadoop.hive.ql.exec.TaskFactory import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.optimizer.Optimizer import org.apache.hadoop.hive.ql.parse._ @@ -40,9 +43,10 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.storage.StorageLevel import shark.{CachedTableRecovery, LogHelper, SharkConfVars, SharkEnv, Utils} -import shark.execution.{HiveDesc, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator, - SharkDDLWork, SparkWork, TerminalOperator} +import shark.execution.{HiveDesc, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator} +import shark.execution.{SharkDDLWork, SparkLoadWork, SparkWork, TerminalOperator} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} +import shark.memstore2.{MemoryTable, PartitionedMemoryTable} /** @@ -62,7 +66,21 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with override def getResultSchema() = _resSchema /** - * Override SemanticAnalyzer.analyzeInternal to handle CTAS caching. + * Override SemanticAnalyzer.analyzeInternal to handle CTAS caching and INSERT updates. + * + * Unified views: + * For CTAS and INSERT INTO/OVERWRITE the generated Shark query plan matches the one + * created if the target table were not cached. Disk => memory loading is done by a + * SparkLoadTask that executes _after_ all other tasks (SparkTask, Hive MoveTasks) finish + * executing. For INSERT INTO, this allows the SparkLoadTask to determine, based on a + * snapshot of the table/partition data directory taken in genMapRedTasks(), and load new + * file content into the cache. For CTAS, everything in the data directory is loaded into + * the cache. + * + * Non-unified views (i.e., the cached table content is memory-only): + * The query plan's FileSinkOperator is replaced by a MemoryStoreSinkOperator. The + * MemoryStoreSinkOperator creates a new table (or partition) entry in the Shark metastore + * for CTAS, and creates UnionRDDs for INSERT INTO commands. */ override def analyzeInternal(ast: ASTNode): Unit = { reset() @@ -123,7 +141,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with logDebug("Completed getting MetaData in Shark Semantic Analysis") // Reset makes sure we don't run the mapred jobs generated by Hive. - if (shouldReset) reset() + if (shouldReset) { + reset() + } // Save the result schema derived from the sink operator produced // by genPlan. This has the correct column names, which clients @@ -150,15 +170,14 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with pCtx.getTopOps().values().head) // TODO: clean the following code. It's too messy to understand... - val terminalOpSeq = { + val terminalOpSeq: Seq[TerminalOperator] = { val qbParseInfo = qb.getParseInfo if (qbParseInfo.isInsertToTable && !qb.isCTAS) { // Handle INSERT. There can be multiple Hive sink operators if the single command comprises // multiple INSERTs. hiveSinkOps.map { hiveSinkOp => - val tableName = hiveSinkOp.asInstanceOf[HiveFileSinkOperator].getConf().getTableInfo() - .getTableName() - + val tableDesc = hiveSinkOp.asInstanceOf[HiveFileSinkOperator].getConf().getTableInfo() + val tableName = tableDesc.getTableName if (tableName == null || tableName == "") { // If table name is empty, it is an INSERT (OVERWRITE) DIRECTORY. OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) @@ -169,31 +188,48 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val databaseName = tableNameSplit(0) if (SharkEnv.memoryMetadataManager.containsTable(databaseName, cachedTableName)) { if (hiveSinkOps.size == 1) { - // If useUnionRDD is false, the sink op is for INSERT OVERWRITE. - val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) val table = SharkEnv.memoryMetadataManager.getTable( databaseName, cachedTableName).get + // INSERT update on a cached table. + qb.targetTableDesc = tableDesc + // If useUnionRDD is true, the sink op is for INSERT INTO. + val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) val cacheMode = table.cacheMode - var hivePartitionKey = new String - val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( + val preferredStorageLevel = table.preferredStorageLevel + val isPartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( databaseName, cachedTableName) - if (isHivePartitioned) { + var hivePartitionKey = if (isPartitioned) { if (cacheMode == CacheType.TACHYON) { throw new SemanticException( "Shark does not support caching Hive-partitioned table(s) in Tachyon.") } - hivePartitionKey = SharkSemanticAnalyzer.getHivePartitionKey(qb) + SharkSemanticAnalyzer.getHivePartitionKey(qb) + } else { + new String + } + if (table.unifyView) { + qb.unifyView = true + qb.targetTableDesc = tableDesc + qb.preferredStorageLevel = preferredStorageLevel + val diskSerDe = table match { + case memoryTable: MemoryTable => memoryTable.diskSerDe + case partitionedTable: PartitionedMemoryTable => { + partitionedTable.getDiskSerDe(hivePartitionKey).getOrElse( + partitionedTable.diskSerDe) + } + } + OperatorFactory.createUnifiedViewFileOutputPlan(hiveSinkOp, diskSerDe) + } else { + OperatorFactory.createSharkMemoryStoreOutputPlan( + hiveSinkOp, + cachedTableName, + databaseName, + preferredStorageLevel, + _resSchema.size, /* numColumns */ + hivePartitionKey, + cacheMode, + useUnionRDD) } - val preferredStorageLevel = table.preferredStorageLevel - OperatorFactory.createSharkMemoryStoreOutputPlan( - hiveSinkOp, - cachedTableName, - databaseName, - preferredStorageLevel, - _resSchema.size, /* numColumns */ - hivePartitionKey, - cacheMode, - useUnionRDD) } else { throw new SemanticException( "Shark does not support updating cached table(s) with multiple INSERTs") @@ -206,26 +242,35 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } else if (hiveSinkOps.size == 1) { // For a single output, we have the option of choosing the output // destination (e.g. CTAS with table property "shark.cache" = "true"). - Seq { - if (qb.isCTAS && qb.getTableDesc != null && - CacheType.shouldCache(qb.getCacheModeForCreateTable())) { - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - qb.getTableDesc().getTblProps.get("shark.cache.storageLevel")) + if (qb.isCTAS && qb.getTableDesc != null && + CacheType.shouldCache(qb.cacheModeForCreateTable)) { + // The table being created from CTAS will be cached. Check whether it should be + // synchronized with disk (i.e., maintain a unified view) or memory-only. + val tblProps = qb.getTableDesc().getTblProps + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + if (qb.unifyView) { + // Save the preferred storage level - needed to create a SparkLoadTask in + // genMapRedTasks(). + // Create the usual Shark file output plan. + qb.preferredStorageLevel = preferredStorageLevel + Seq(OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head)) + } else { qb.getTableDesc().getTblProps().put(CachedTableRecovery.QUERY_STRING, ctx.getCmd()) - OperatorFactory.createSharkMemoryStoreOutputPlan( + Seq(OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOps.head, qb.getTableDesc.getTableName, qb.getTableDesc.getDatabaseName, preferredStorageLevel, _resSchema.size, /* numColumns */ new String, /* hivePartitionKey */ - qb.getCacheModeForCreateTable, - false /* useUnionRDD */) - } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { - OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) - } else { - OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head) + qb.cacheModeForCreateTable, + false /* useUnionRDD */)) } + } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { + Seq(OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head)) + } else { + Seq(OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head)) } // A hack for the query plan dashboard to get the query plan. This was @@ -248,8 +293,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with * Generate tasks for executing the query, including the SparkTask to do the * select, the MoveTask for updates, and the DDLTask for CTAS. */ - def genMapRedTasks(qb: QB, pctx: ParseContext, terminalOps: Seq[TerminalOperator]) { - + def genMapRedTasks(qb: QueryBlock, pctx: ParseContext, terminalOps: Seq[TerminalOperator]) { // Create the spark task. terminalOps.foreach { terminalOp => val task = TaskFactory.get(new SparkWork(pctx, terminalOp, _resSchema), conf) @@ -257,6 +301,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } if (qb.getIsQuery) { + // Note: CTAS isn't considered a query - it's handled in the 'else' block below. // Configure FetchTask (used for fetching results to CLIDriver). val loadWork = getParseContext.getLoadFileWork.get(0) val cols = loadWork.getColumns @@ -272,11 +317,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with setFetchTask(fetchTask) } else { - // Configure MoveTasks for table updates (e.g. CTAS, INSERT). + // Configure MoveTasks for table updates (CTAS, INSERT). val mvTasks = new ArrayList[MoveTask]() + // For CTAS, 'fileWork' contains a single LoadFileDesc (called "LoadFileWork" in Hive). val fileWork = getParseContext.getLoadFileWork - // Used for loading non-native tables. val tableWork = getParseContext.getLoadTableWork tableWork.foreach { ltd => mvTasks.add(TaskFactory.get( @@ -285,12 +330,13 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with fileWork.foreach { lfd => if (qb.isCTAS) { + // For CTAS, lfd.targetDir should be the data directory of the table being created. var location = qb.getTableDesc.getLocation if (location == null) { try { - val dumpTable = db.newTable(qb.getTableDesc.getTableName) + val tableToCreate = db.newTable(qb.getTableDesc.getTableName) val wh = new Warehouse(conf) - location = wh.getTablePath(db.getDatabase(dumpTable.getDbName()), dumpTable + location = wh.getTablePath(db.getDatabase(tableToCreate.getDbName()), tableToCreate .getTableName()).toString; } catch { case e: HiveException => throw new SemanticException(e) @@ -304,14 +350,13 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with new MoveWork(null, null, null, lfd, false), conf).asInstanceOf[MoveTask]) } - // The move task depends on all root tasks. In the case of multi outputs, + // The move task depends on all root tasks. In the case of multiple outputs, // the moves are only started once all outputs are executed. // Note: For a CTAS for a memory-only cached table, a MoveTask is still added as a child of // the main SparkTask. However, there no effects from its execution, since the SELECT query // output is piped to Shark's in-memory columnar storage builder, instead of a Hive tmp // directory. // TODO(harvey): Don't create a MoveTask in this case. - val hiveFileSinkOp = terminalOps.head.localHiveOp mvTasks.foreach { moveTask => rootTasks.foreach { rootTask => rootTask.addDependentTask(moveTask) @@ -331,6 +376,54 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } */ } + + if (qb.unifyView) { + val sparkLoadWork = if (qb.isCTAS) { + new SparkLoadWork( + qb.createTableDesc.getDatabaseName, + qb.createTableDesc.getTableName, + partSpecOpt = None, + SparkLoadWork.CommandTypes.NEW_ENTRY, + qb.preferredStorageLevel, + pathFilter = None) + } else { + // Split from 'databaseName.tableName' + val tableNameSplit = qb.targetTableDesc.getTableName.split('.') + val databaseName = tableNameSplit(0) + val cachedTableName = tableNameSplit(1) + val hiveTable = db.getTable(databaseName, cachedTableName) + val destPartition = qb.getMetaData.getDestPartitionForAlias( + qb.getParseInfo.getClauseNamesForDest.head) + val partitionSpec = if (destPartition == null) null else destPartition.getSpec + val isInsertInto = qb.getParseInfo.isInsertIntoTable(cachedTableName) + val (insertType, pathFilterOpt) = + if (isInsertInto) { + val pathOpt = if (hiveTable.isPartitioned) { + val partitionOpt = Option( + db.getPartition(hiveTable, partitionSpec, false /* forceCreate */)) + partitionOpt.map(_.getPartitionPath) + } else { + Some(hiveTable.getPath) + } + (SparkLoadWork.CommandTypes.INSERT, pathOpt.map(Utils.createSnapshotFilter(_, conf))) + } else { + (SparkLoadWork.CommandTypes.OVERWRITE, None) + } + + // Add a SparkLoadTask as a dependent of all MoveTasks, so that when executed, the table's + // (or table partition's) data directory will contain the updates that need to be loaded + // into memory. + new SparkLoadWork( + databaseName, + cachedTableName, + Option(partitionSpec), + insertType, + qb.preferredStorageLevel, + pathFilterOpt) + } + val sparkLoadTask = TaskFactory.get(sparkLoadWork, conf) + mvTasks.foreach(_.addDependentTask(sparkLoadTask)) + } } // For CTAS, generate a DDL task to create the table. This task should be a @@ -389,6 +482,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with getParseContext.getQB.getTableDesc } + // Update the QueryBlock passed into this method. + // TODO(harvey): Remove once the TODO above is fixed. + queryBlock.setTableDesc(createTableDesc) + // 'createTableDesc' is NULL if there is an IF NOT EXISTS condition and the target table // already exists. if (createTableDesc != null) { @@ -414,7 +511,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } val shouldCache = CacheType.shouldCache(cacheMode) - if (shouldCache) { + queryBlock.unifyView = shouldCache && + createTableProperties.getOrElse("shark.cache.unifyView", + SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean + createTableProperties.put("shark.cache.unifyView", queryBlock.unifyView.toString) + if (shouldCache && !queryBlock.unifyView) { createTableDesc.setSerName(classOf[ColumnarSerDe].getName) } @@ -433,7 +534,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with hiveDDLTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } - queryBlock.setCacheModeForCreateTable(cacheMode) + queryBlock.cacheModeForCreateTable = cacheMode queryBlock.setTableDesc(createTableDesc) } return queryStmtASTNode From a4a4527faaaf31820d59bc192aeed295e2efd34d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 6 Nov 2013 00:23:39 -0800 Subject: [PATCH 219/331] Misc cleanup/adjustments from unified view changes. --- src/main/scala/shark/SharkEnv.scala | 3 ++- .../scala/shark/api/RDDTableFunctions.scala | 4 +-- .../execution/MemoryStoreSinkOperator.scala | 2 +- .../shark/execution/TableScanOperator.scala | 27 ++++++------------- .../scala/shark/memstore2/ColumnarSerDe.scala | 2 +- src/test/scala/shark/SQLSuite.scala | 8 +++--- 6 files changed, 18 insertions(+), 28 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index c8b28706..0398c439 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -116,7 +116,8 @@ object SharkEnv extends LogHelper { * delegating to MemoryMetadataManager#removeTable() for removing the table's entry from the * Shark metastore. * - * @param tableName The table that should be dropped from the Shark metastore and from memory storage. + * @param tableName The table that should be dropped from the Shark metastore and from memory + * storage. */ def dropTable(databaseName: String, tableName: String): Option[RDD[_]] = { val tableKey = makeTachyonTableKey(databaseName, tableName) diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 43c686a6..a5fb7866 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -30,7 +30,7 @@ import shark.util.HiveUtils class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { - def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { + def saveAsTable(tableName: String, fields: Seq[String], unifyView: Boolean = false): Boolean = { require(fields.size == this.manifests.size, "Number of column names != number of fields in the RDD.") @@ -63,7 +63,7 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { if (isSucessfulCreateTable) { // Create an entry in the MemoryMetadataManager. val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, CacheType.HEAP, rdd.getStorageLevel) + databaseName, tableName, CacheType.HEAP, rdd.getStorageLevel, unifyView) newTable.tableRDD = rdd try { // Force evaluate to put the data in memory. diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 5d4d63ab..6a4be1d1 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -204,7 +204,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, cacheMode, storageLevel)) + databaseName, tableName, cacheMode, storageLevel, unifyView = false)) memoryTable.tableRDD = outputRDD } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index bca3ddc8..05a93be5 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -38,6 +38,7 @@ import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.optimization.ColumnPruner import shark.memstore2.{CacheType, MemoryMetadataManager, TablePartition, TablePartitionStats} +import shark.util.HiveUtils /** @@ -79,23 +80,9 @@ class TableScanOperator extends TopOperator[TableScanDesc] { serializer.getObjectInspector() } else { val partProps = firstConfPartDesc.getProperties() - val tableDeser = firstConfPartDesc.getDeserializerClass().newInstance() - tableDeser.initialize(hconf, partProps) - val partCols = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - val partNames = new ArrayList[String] - val partObjectInspectors = new ArrayList[ObjectInspector] - partCols.trim().split("/").foreach { key => - partNames.add(key) - partObjectInspectors.add(PrimitiveObjectInspectorFactory.javaStringObjectInspector) - } - - val partObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( - partNames, partObjectInspectors) - val oiList = Arrays.asList( - tableDeser.getObjectInspector().asInstanceOf[StructObjectInspector], - partObjectInspector.asInstanceOf[StructObjectInspector]) - // new oi is union of table + partition object inspectors - ObjectInspectorFactory.getUnionStructObjectInspector(oiList) + val partSerDe = firstConfPartDesc.getDeserializerClass().newInstance() + partSerDe.initialize(hconf, partProps) + HiveUtils.makeUnionOIForPartitionedTable(partProps, partSerDe) } } @@ -137,8 +124,10 @@ class TableScanOperator extends TopOperator[TableScanDesc] { private def createPrunedRdd(databaseName: String, tableName: String, rdd: RDD[_]): RDD[_] = { // Stats used for map pruning. - val indexToStats: collection.Map[Int, TablePartitionStats] = - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName).get + val indexToStatsOpt: Option[collection.Map[Int, TablePartitionStats]] = + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) + assert (indexToStatsOpt.isDefined, "Stats not found for table " + tableName) + val indexToStats = indexToStatsOpt.get // Run map pruning if the flag is set, there exists a filter predicate on // the input table and we have statistics on the table. diff --git a/src/main/scala/shark/memstore2/ColumnarSerDe.scala b/src/main/scala/shark/memstore2/ColumnarSerDe.scala index 4c8bef76..79c6f282 100644 --- a/src/main/scala/shark/memstore2/ColumnarSerDe.scala +++ b/src/main/scala/shark/memstore2/ColumnarSerDe.scala @@ -51,7 +51,7 @@ class ColumnarSerDe extends SerDe with LogHelper { objectInspector = ColumnarStructObjectInspector(serDeParams) // This null check is needed because Hive's SemanticAnalyzer.genFileSinkPlan() creates - // an instance of the table's StructObjectInspector by creating an instance SerDe, which + // an instance of the table's StructObjectInspector by creating an instance of SerDe, which // it initializes by passing a 'null' argument for 'conf'. if (conf != null) { var partitionSize = { diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 85d671b1..7fffab92 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -887,11 +887,11 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe. - HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, columnarSerDeName, hiveConf) + HiveUtils.alterSerdeInHive(tableName, None /* partitionSpecOpt */, columnarSerDeName, hiveConf) assert(getTableSerDeName(tableName) == columnarSerDeName) // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive(tableName, null /* partitionSpec */, oldSerDeName, hiveConf) + HiveUtils.alterSerdeInHive(tableName, None /* partitionSpecOpt */, oldSerDeName, hiveConf) assert(getTableSerDeName(tableName) == oldSerDeName) } @@ -915,11 +915,11 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe - HiveUtils.alterSerdeInHive(tableName, partitionSpec, columnarSerDeName, hiveConf) + HiveUtils.alterSerdeInHive(tableName, Some(partitionSpec), columnarSerDeName, hiveConf) assert(getPartitionSerDeName(tableName, partitionSpec) == columnarSerDeName) // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive(tableName, partitionSpec, oldSerDeName, hiveConf) + HiveUtils.alterSerdeInHive(tableName, Some(partitionSpec), oldSerDeName, hiveConf) assert(getPartitionSerDeName(tableName, partitionSpec) == oldSerDeName) } } From 7d475f604fd28b5c0a35277f9aa3db8eb9df577c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 6 Nov 2013 00:25:20 -0800 Subject: [PATCH 220/331] Alter SerDes for regular CREATE TABLE commands for tables with unified views. The SharkDDLTask alters the SerDe by creating an launching a Hive DDLTask. This breaks Hive abstraction, but is much simpler to implement... --- .../scala/shark/execution/SharkDDLTask.scala | 25 ++++++++++++++----- 1 file changed, 19 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 161a586a..ddc19077 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -30,7 +30,8 @@ import org.apache.hadoop.hive.ql.plan.api.StageType import org.apache.spark.rdd.EmptyRDD import shark.{LogHelper, SharkConfVars, SharkEnv} -import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} +import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager, PartitionedMemoryTable} +import shark.util.HiveUtils private[shark] class SharkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { @@ -57,7 +58,6 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] // TODO(harvey): Check whether the `hiveDb` is needed. HiveTask should already have a `db` to // use. - work.ddlDesc match { case creatTblDesc: CreateTableDesc => createTable(hiveDb, creatTblDesc, work.cacheMode) case addPartitionDesc: AddPartitionDesc => addPartition(hiveDb, addPartitionDesc) @@ -84,8 +84,10 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( tblProps.get("shark.cache.storageLevel")) + val unifyView = tblProps.getOrElse("shark.cache.unifyView", + SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean val isHivePartitioned = (createTblDesc.getPartCols.size > 0) - if (isHivePartitioned) { + val newTable = if (isHivePartitioned) { // Add a new PartitionedMemoryTable entry in the Shark metastore. // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( @@ -93,12 +95,23 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] tableName, cacheMode, preferredStorageLevel, + unifyView, tblProps) } else { - val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - dbName, tableName, cacheMode, preferredStorageLevel) + val memoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( + dbName, tableName, cacheMode, preferredStorageLevel, unifyView) // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. - newTable.tableRDD = new EmptyRDD(SharkEnv.sc) + memoryTable.tableRDD = new EmptyRDD(SharkEnv.sc) + memoryTable + } + if (unifyView) { + val table = hiveMetadataDb.getTable(tableName) + newTable.diskSerDe = table.getDeserializer().getClass.getName + HiveUtils.alterSerdeInHive( + tableName, + partitionSpecOpt = None, + classOf[ColumnarSerDe].getName, + conf) } } From 1aaec63718cca57c46bf2178452af0cfdbb57360 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 6 Nov 2013 00:25:50 -0800 Subject: [PATCH 221/331] Flatten both RDD arguments for RDDUtils#unionAndFlatten() --- src/main/scala/shark/execution/RDDUtils.scala | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 7d18dd53..e102497b 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -33,16 +33,22 @@ import shark.SharkEnv */ object RDDUtils { + /** + * Returns a UnionRDD using both RDD arguments. Any UnionRDD argument is "flattened", in that + * its parent sequence of RDDs are used to compose the returned UnionRDD. + */ def unionAndFlatten[T: ClassManifest]( rdd: RDD[T], - otherRdd: RDD[T]): RDD[T] = { - val unionedRdd = otherRdd match { - case unionRdd: UnionRDD[_] => { - new UnionRDD(rdd.context, (unionRdd.rdds :+ rdd)) - } - case _ => rdd.union(otherRdd) + otherRdd: RDD[T]): UnionRDD[T] = { + val otherRdds = otherRdd match { + case otherUnionRdd: UnionRDD[_] => otherUnionRdd.rdds + case _ => Seq(otherRdd) + } + val rdds = rdd match { + case unionRdd: UnionRDD[_] => unionRdd.rdds + case _ => Seq(rdd) } - return unionedRdd + new UnionRDD(rdd.context, rdds ++ otherRdds) } def unpersistRDD(rdd: RDD[_]): RDD[_] = { From d467482af797b128a0715fb41de0511e944f9f41 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 6 Nov 2013 04:55:43 -0800 Subject: [PATCH 222/331] Non-working version of table persistence. --- src/main/scala/shark/SharkEnv.scala | 1 + .../memstore2/MemoryMetadataManager.scala | 38 ++++++++++++++++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 0398c439..1d9a07d1 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -134,6 +134,7 @@ object SharkEnv extends LogHelper { /** Cleans up and shuts down the Shark environments. */ def stop() { logDebug("Shutting down Shark Environment") + memoryMetadataManager.shutdown() // Stop the SparkContext if (SharkEnv.sc != null) { sc.stop() diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 950df71e..1f2fdea3 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -21,7 +21,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.{HashMap => JavaHashMap, Map => JavaMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.ConcurrentMap +import scala.collection.mutable.{ArrayBuffer, ConcurrentMap} import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.storage.StorageLevel @@ -29,6 +29,7 @@ import org.apache.spark.storage.StorageLevel import shark.execution.RDDUtils import shark.SharkConfVars import shark.SharkEnv +import shark.util.HiveUtils class MemoryMetadataManager { @@ -40,6 +41,9 @@ class MemoryMetadataManager { private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] + // List of callback functions to execute when the Shark metastore shuts down. + private val _onShutdownCallbacks = new ArrayBuffer[() => Unit] + def putStats( databaseName: String, tableName: String, @@ -163,6 +167,29 @@ class MemoryMetadataManager { return tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) } + def shutdown() { + resetUnifiedTableSerdes() + } + + def resetUnifiedTableSerdes() { + for (table <- _keyToTable.values.filter(_.unifyView)) { + table match { + case memoryTable: MemoryTable => { + HiveUtils.alterSerdeInHive( + memoryTable.tableName, + None /* partitionSpecOpt */, + memoryTable.diskSerDe) + } + case partitionedTable: PartitionedMemoryTable => { + for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { + val partitionSpec = MemoryMetadataManager.parseHivePartitionKeyStr(hiveKeyStr) + HiveUtils.alterSerdeInHive(partitionedTable.tableName, Some(partitionSpec), serDeName) + } + } + } + } + } + /** * Find all keys that are strings. Used to drop tables after exiting. * @@ -209,6 +236,15 @@ object MemoryMetadataManager { return keyStr } + def parseHivePartitionKeyStr(keyStr: String): JavaMap[String, String] = { + val partitionSpec = new JavaHashMap[String, String]() + for (pair <- keyStr.split("/")) { + val pairSplit = pair.split("=") + partitionSpec.put(pairSplit(0), pairSplit(1)) + } + partitionSpec + } + /** Return a StorageLevel corresponding to its String name. */ def getStorageLevelFromString(s: String): StorageLevel = { if (s == null || s == "") { From 5f0f91215b33c38c760c0e5c45aad26277e382a8 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 7 Nov 2013 18:57:37 -0800 Subject: [PATCH 223/331] Comment about the unify view creation in SharkDDLTask. --- src/main/scala/shark/execution/RDDUtils.scala | 2 +- src/main/scala/shark/execution/SharkDDLTask.scala | 4 ++++ src/main/scala/shark/execution/SparkLoadTask.scala | 6 ++---- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index e102497b..28dcf62d 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -35,7 +35,7 @@ object RDDUtils { /** * Returns a UnionRDD using both RDD arguments. Any UnionRDD argument is "flattened", in that - * its parent sequence of RDDs are used to compose the returned UnionRDD. + * its parent sequence of RDDs is directly passed to the UnionRDD returned. */ def unionAndFlatten[T: ClassManifest]( rdd: RDD[T], diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index ddc19077..f7989a23 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -107,6 +107,10 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] if (unifyView) { val table = hiveMetadataDb.getTable(tableName) newTable.diskSerDe = table.getDeserializer().getClass.getName + // This creates and directly executes a Hive DDLTask to change the table's SerDe property in + // the Hive metastore. + // The alternatives are to either attach a HiveDDLTask dependent to this SharkDDLTask or to + // copy the necessary code from Hive. HiveUtils.alterSerdeInHive( tableName, partitionSpecOpt = None, diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 1de575fd..a1003fc2 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -110,6 +110,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } + // TODO(harvey): Multiple partition specs... case partitionedTable: PartitionedMemoryTable => { val partCols = hiveTable.getPartCols.map(_.getName) val partSpecs = work.partSpecOpt.get @@ -176,10 +177,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } work.partSpecOpt match { case Some(partSpecs) => { - val hivePartition = Hive.get(conf).getPartition( - hiveTable, - partSpecs, - false /* forceCreate */) + val hivePartition = Hive.get(conf).getPartition(hiveTable, partSpecs, false /* forceCreate */) val partSerDe = hivePartition.getDeserializer val partCols = hiveTable.getPartCols.map(_.getName) val partSchema = hivePartition.getSchema From 5f6ccca694db15586e3f4ad4be199f0887a6193b Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 8 Nov 2013 14:46:16 -0800 Subject: [PATCH 224/331] SparkLoadTask cleanup. --- .../scala/shark/execution/SparkLoadTask.scala | 329 ++++++++++-------- 1 file changed, 181 insertions(+), 148 deletions(-) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index a1003fc2..1d8aad46 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -73,159 +73,23 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val databaseName = work.databaseName val tableName = work.tableName val hiveTable = Hive.get(conf).getTable(databaseName, tableName) - val tableDesc = Utilities.getTableDesc(hiveTable) val oi = hiveTable.getDeserializer().getObjectInspector().asInstanceOf[StructObjectInspector] - val sharkTableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) - val hadoopReader = new HadoopTableReader(tableDesc, conf) + val hadoopReader = new HadoopTableReader(Utilities.getTableDesc(hiveTable), conf) - // TODO(harvey): Cleanup. This part overlaps with execution code in MemoryStoreSinkOperator. - sharkTableOpt match { - case Some(sharkTable) => { - var unionWithPreviousStats = false - sharkTable match { - case memoryTable: MemoryTable => { - val serDe = Class.forName( - memoryTable.diskSerDe).newInstance.asInstanceOf[Deserializer] - serDe.initialize(conf, tableDesc.getProperties) - val inputRDD = hadoopReader.makeRDDForTable( - hiveTable, - work.pathFilter, - serDe.getClass) - val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( - inputRDD, - tableDesc.getProperties, - work.storageLevel, - hadoopReader.broadcastedHiveConf, - serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) - memoryTable.tableRDD = work.commandType match { - case SparkLoadWork.CommandTypes.OVERWRITE => tablePartitionRDD - case SparkLoadWork.CommandTypes.INSERT => { - RDDUtils.unionAndFlatten(tablePartitionRDD, memoryTable.tableRDD) - } - case SparkLoadWork.CommandTypes.NEW_ENTRY => { - throw new Exception( - "Invalid state: table %s already exists in memory".format(tableName)) - } - } - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) - } - // TODO(harvey): Multiple partition specs... - case partitionedTable: PartitionedMemoryTable => { - val partCols = hiveTable.getPartCols.map(_.getName) - val partSpecs = work.partSpecOpt.get - val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpecs) - val partition = db.getPartition(hiveTable, partSpecs, false /* forceCreate */) - val partSerDe = Class.forName(partitionedTable.getDiskSerDe(partitionKey).getOrElse( - partitionedTable.diskSerDe)).newInstance.asInstanceOf[Deserializer] - val partSchema = partition.getSchema - partSerDe.initialize(conf, partSchema) - val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) - val inputRDD = hadoopReader.makeRDDForPartitionedTable( - Map(partition -> partSerDe.getClass), work.pathFilter) - val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( - inputRDD, - addPartitionInfoToSerDeProps(partCols, new Properties(partition.getSchema)), - work.storageLevel, - hadoopReader.broadcastedHiveConf, - unionOI) - partitionedTable.getPartition(partitionKey) match { - case Some(previousRDD) => { - work.commandType match { - case SparkLoadWork.CommandTypes.OVERWRITE => { - partitionedTable.putPartition(partitionKey, tablePartitionRDD) - } - case SparkLoadWork.CommandTypes.INSERT => { - partitionedTable.updatePartition( - partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) - // Union stats for the previous RDD with the new RDD loaded. - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { - case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) - case _ => Unit - } - } - case SparkLoadWork.CommandTypes.NEW_ENTRY => { - throw new Exception( - "Invalid state: table %s already exists in memory".format(tableName)) - } - } - } - case None => { - // The partition being updated does not currently exist. Create and set a new - // partition key entry, and register a shutdown callback in the Shark metastore. - partitionedTable.putPartition(partitionKey, tablePartitionRDD) - } - } - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) - } - } + work.partSpecOpt match { + case Some(partSpec) => { + loadPartitionedTable( + hiveTable, + partSpec, + hadoopReader, + work.pathFilter) } case None => { - // Couldn't find the target table for a LOAD. - assert(work.commandType == SparkLoadWork.CommandTypes.NEW_ENTRY) - - // Add the table to-be-cached to the Shark metastore. - val tblProps = hiveTable.getParameters() - val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) - val inputRDD = if (hiveTable.isPartitioned) { - val partition = db.getPartition(hiveTable, work.partSpecOpt.get, false /* forceCreate */) - hadoopReader.makeRDDForPartitionedTable(Seq(partition)) - } else { - hadoopReader.makeRDDForTable(hiveTable) - } - work.partSpecOpt match { - case Some(partSpecs) => { - val hivePartition = Hive.get(conf).getPartition(hiveTable, partSpecs, false /* forceCreate */) - val partSerDe = hivePartition.getDeserializer - val partCols = hiveTable.getPartCols.map(_.getName) - val partSchema = hivePartition.getSchema - partSerDe.initialize(conf, partSchema) - val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) - // Partitioned table. - val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( - inputRDD, - addPartitionInfoToSerDeProps(partCols, new Properties(hivePartition.getSchema)), - work.storageLevel, - hadoopReader.broadcastedHiveConf, - unionOI) - val newTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( - databaseName, - tableName, - cacheMode, - preferredStorageLevel, - unifyView = true, - tblProps) - val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr( - hiveTable.getPartCols.map(_.getName), partSpecs) - newTable.putPartition(partitionKey, tablePartitionRDD) - newTable.setDiskSerDe(partitionKey, hivePartition.getDeserializer.getClass.getName) - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) - } - case None => { - val serDe = hiveTable.getDeserializer - val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( - inputRDD, - tableDesc.getProperties, - work.storageLevel, - hadoopReader.broadcastedHiveConf, - serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) - // Non-partitioned table. - val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, cacheMode, preferredStorageLevel, unifyView = true) - newTable.tableRDD = tablePartitionRDD - // Record what the previous SerDe was. - newTable.diskSerDe = serDe.getClass.getName - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) - } - } - // Set the new SerDe to be a ColumnarSerDe, since the data has been cached. - HiveUtils.alterSerdeInHive( - tableName, - work.partSpecOpt, - classOf[ColumnarSerDe].getName, - conf) + loadMemoryTable( + hiveTable, + hadoopReader, + work.pathFilter) } } // Success! @@ -263,6 +127,175 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe (transformedRdd, statsAcc.value) } + def createMemoryTable( + databaseName: String, + tableName: String, + preferredStorageLevel: StorageLevel, + defaultDiskSerDe: String, + tblProps: JavaMap[String, String]): MemoryTable = { + val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) + val newMemoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( + databaseName, tableName, cacheMode, preferredStorageLevel, unifyView = true) + newMemoryTable.diskSerDe = defaultDiskSerDe + HiveUtils.alterSerdeInHive( + tableName, + partitionSpecOpt = None, + classOf[ColumnarSerDe].getName, + conf) + newMemoryTable + } + + def loadMemoryTable( + hiveTable: HiveTable, + hadoopReader: HadoopTableReader, + pathFilter: Option[PathFilter]) { + val databaseName = hiveTable.getDbName + val tableName = hiveTable.getTableName + val tblProps = hiveTable.getParameters + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + val memoryTable = work.commandType match { + case SparkLoadWork.CommandTypes.NEW_ENTRY => { + createMemoryTable( + databaseName, + tableName, + preferredStorageLevel, + hiveTable.getDeserializer.getClass.getName, + tblProps) + } + case _ => { + SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { + case Some(table: MemoryTable) => table + case _ => { + throw new Exception("Invalid state: cached table being updated doesn't exist.") + } + } + } + } + val tableSchema = hiveTable.getSchema + val serDe = Class.forName(memoryTable.diskSerDe).newInstance.asInstanceOf[Deserializer] + serDe.initialize(conf, tableSchema) + val inputRDD = hadoopReader.makeRDDForTable( + hiveTable, + pathFilter, + serDe.getClass) + val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + inputRDD, + tableSchema, + work.storageLevel, + hadoopReader.broadcastedHiveConf, + serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) + memoryTable.tableRDD = work.commandType match { + case (SparkLoadWork.CommandTypes.OVERWRITE + | SparkLoadWork.CommandTypes.NEW_ENTRY) => tablePartitionRDD + case SparkLoadWork.CommandTypes.INSERT => { + val unionedRDD = RDDUtils.unionAndFlatten(tablePartitionRDD, memoryTable.tableRDD) + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName ) match { + case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) + case None => Unit + } + unionedRDD + } + } + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) + } + + def createPartitionedTable( + databaseName: String, + tableName: String, + preferredStorageLevel: StorageLevel, + defaultDiskSerDe: String, + tblProps: JavaMap[String, String], + partSpecs: JavaMap[String, String]): PartitionedMemoryTable = { + val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) + val newPartitionedTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( + databaseName, + tableName, + cacheMode, + preferredStorageLevel, + unifyView = true, + tblProps) + newPartitionedTable.diskSerDe = defaultDiskSerDe + HiveUtils.alterSerdeInHive( + tableName, + Some(partSpecs), + classOf[ColumnarSerDe].getName, + conf) + newPartitionedTable + } + + def loadPartitionedTable( + hiveTable: HiveTable, + partSpecs: JavaMap[String, String], + hadoopReader: HadoopTableReader, + pathFilter: Option[PathFilter]) { + // TODO(harvey): Multiple partition specs... + val databaseName = hiveTable.getDbName + val tableName = hiveTable.getTableName + val tblProps = hiveTable.getParameters + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + val partitionedTable = work.commandType match { + case SparkLoadWork.CommandTypes.NEW_ENTRY => { + createPartitionedTable( + databaseName, + tableName, + preferredStorageLevel, + hiveTable.getDeserializer.getClass.getName, + tblProps, + partSpecs) + } + case _ => { + SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { + case Some(table: PartitionedMemoryTable) => table + case _ => { + throw new Exception("Invalid state: cached table being updated doesn't exist.") + } + } + } + } + val partCols = hiveTable.getPartCols.map(_.getName) + val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpecs) + val partition = db.getPartition(hiveTable, partSpecs, false /* forceCreate */) + val partSerDe = Class.forName(partitionedTable.getDiskSerDe(partitionKey).getOrElse( + partitionedTable.diskSerDe)).newInstance.asInstanceOf[Deserializer] + val partSchema = partition.getSchema + partSerDe.initialize(conf, partSchema) + val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) + val inputRDD = hadoopReader.makeRDDForPartitionedTable( + Map(partition -> partSerDe.getClass), pathFilter) + val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + inputRDD, + addPartitionInfoToSerDeProps(partCols, new Properties(partition.getSchema)), + preferredStorageLevel, + hadoopReader.broadcastedHiveConf, + unionOI) + work.commandType match { + case SparkLoadWork.CommandTypes.OVERWRITE | SparkLoadWork.CommandTypes.NEW_ENTRY => { + partitionedTable.putPartition(partitionKey, tablePartitionRDD) + } + case SparkLoadWork.CommandTypes.INSERT => { + val previousRDD = partitionedTable.getPartition(partitionKey) match { + case Some(previousRDD) => { + partitionedTable.updatePartition( + partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) + // Note: these matches have to be separate, since an empty partition is represented by + // an empty RDD. If it's already cached in memory, then + // PartitionedMemoryTable#updatePartition() must be called. + // Union stats for the previous RDD with the new RDD loaded. + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName ) match { + case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) + case None => Unit + } + } + case None => partitionedTable.putPartition(partitionKey, tablePartitionRDD) + } + } + } + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) + } + + def unionStatsMaps( targetStatsMap: ArrayBuffer[(Int, TablePartitionStats)], otherStatsMap: Iterable[(Int, TablePartitionStats)] From cfe461988460100764503bb0ed6d6f020aa04997 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 8 Nov 2013 18:10:22 -0800 Subject: [PATCH 225/331] Don't pass StorageLevel to SparkLoadTask. It should be accessible through Hive table metadata properties. --- src/main/scala/shark/execution/SparkLoadTask.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 1d8aad46..52a198a5 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -53,7 +53,6 @@ class SparkLoadWork( val tableName: String, val partSpecOpt: Option[JavaMap[String, String]], val commandType: SparkLoadWork.CommandTypes.Type, - val storageLevel: StorageLevel, val pathFilter: Option[PathFilter]) extends java.io.Serializable @@ -182,7 +181,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( inputRDD, tableSchema, - work.storageLevel, + preferredStorageLevel, hadoopReader.broadcastedHiveConf, serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) memoryTable.tableRDD = work.commandType match { From 092244fea2896d4aa7dbc6433c621bc4ef00da3b Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 8 Nov 2013 19:06:07 -0800 Subject: [PATCH 226/331] No need to pass storage levels to ddl tasks in semantic analysis. --- src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala | 2 -- src/main/scala/shark/parse/SharkSemanticAnalyzer.scala | 2 -- 2 files changed, 4 deletions(-) diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index 89002734..40800032 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -46,7 +46,6 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con if (tableOpt.exists(_.unifyView)) { // Find the arguments needed to instantiate a SparkLoadWork. val tableSpec = new BaseSemanticAnalyzer.tableSpec(db, conf, tableASTNode) - val preferredStorageLevel = tableOpt.get.preferredStorageLevel val hiveTable = tableSpec.tableHandle val partSpecOpt = Option(tableSpec.getPartSpec()) val dataPath = if (partSpecOpt.isEmpty) { @@ -77,7 +76,6 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con tableName, partSpecOpt, loadCommandType, - preferredStorageLevel, Some(fileFilter)) moveTask.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 8f73126c..c1ee6016 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -384,7 +384,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.createTableDesc.getTableName, partSpecOpt = None, SparkLoadWork.CommandTypes.NEW_ENTRY, - qb.preferredStorageLevel, pathFilter = None) } else { // Split from 'databaseName.tableName' @@ -418,7 +417,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with cachedTableName, Option(partitionSpec), insertType, - qb.preferredStorageLevel, pathFilterOpt) } val sparkLoadTask = TaskFactory.get(sparkLoadWork, conf) From a8fb6c5ab4884e402f44237584f9b14d39e31a39 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 8 Nov 2013 19:21:34 -0800 Subject: [PATCH 227/331] Remove getAllKeyStrings() from MemoryMetadataManager. --- .../shark/memstore2/MemoryMetadataManager.scala | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 1f2fdea3..36c5c570 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -23,6 +23,8 @@ import java.util.{HashMap => JavaHashMap, Map => JavaMap} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, ConcurrentMap} +import org.apache.hadoop.hive.ql.metadata.Hive + import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.storage.StorageLevel @@ -175,6 +177,8 @@ class MemoryMetadataManager { for (table <- _keyToTable.values.filter(_.unifyView)) { table match { case memoryTable: MemoryTable => { + // Reset unified table SerDes, so that next time we start up in Shark, those tables can + // be read from disk without special handling. HiveUtils.alterSerdeInHive( memoryTable.tableName, None /* partitionSpecOpt */, @@ -190,15 +194,6 @@ class MemoryMetadataManager { } } - /** - * Find all keys that are strings. Used to drop tables after exiting. - * - * TODO(harvey): Won't be needed after unifed views are added. - */ - def getAllKeyStrings(): Seq[String] = { - _keyToTable.keys.collect { case k: String => k } toSeq - } - private def makeTableKey(databaseName: String, tableName: String): String = { (databaseName + '.' + tableName).toLowerCase } From 43ffeb6bd8862af232e84b9010b8c2424b864eec Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 8 Nov 2013 19:34:49 -0800 Subject: [PATCH 228/331] Some more cleanup in SparkLoadTask. --- .../scala/shark/execution/SharkDDLTask.scala | 1 - .../scala/shark/execution/SparkLoadTask.scala | 162 +++++++++--------- 2 files changed, 85 insertions(+), 78 deletions(-) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index f7989a23..0264aeb5 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -81,7 +81,6 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val dbName = hiveMetadataDb.getCurrentDatabase() val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( tblProps.get("shark.cache.storageLevel")) val unifyView = tblProps.getOrElse("shark.cache.unifyView", diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 52a198a5..e47a9976 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -126,51 +126,52 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe (transformedRdd, statsAcc.value) } - def createMemoryTable( + def getOrCreateMemoryTable( databaseName: String, tableName: String, + cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, defaultDiskSerDe: String, tblProps: JavaMap[String, String]): MemoryTable = { - val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) - val newMemoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, cacheMode, preferredStorageLevel, unifyView = true) - newMemoryTable.diskSerDe = defaultDiskSerDe - HiveUtils.alterSerdeInHive( - tableName, - partitionSpecOpt = None, - classOf[ColumnarSerDe].getName, - conf) - newMemoryTable - } - - def loadMemoryTable( - hiveTable: HiveTable, - hadoopReader: HadoopTableReader, - pathFilter: Option[PathFilter]) { - val databaseName = hiveTable.getDbName - val tableName = hiveTable.getTableName - val tblProps = hiveTable.getParameters - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) - val memoryTable = work.commandType match { + work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { - createMemoryTable( - databaseName, + val newMemoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( + databaseName, tableName, cacheMode, preferredStorageLevel, unifyView = true) + newMemoryTable.diskSerDe = defaultDiskSerDe + HiveUtils.alterSerdeInHive( tableName, - preferredStorageLevel, - hiveTable.getDeserializer.getClass.getName, - tblProps) + partitionSpecOpt = None, + classOf[ColumnarSerDe].getName, + conf) + newMemoryTable } case _ => { SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { case Some(table: MemoryTable) => table case _ => { - throw new Exception("Invalid state: cached table being updated doesn't exist.") + throw new Exception("Internal error: cached table being updated doesn't exist.") } } } } + } + + def loadMemoryTable( + hiveTable: HiveTable, + hadoopReader: HadoopTableReader, + pathFilter: Option[PathFilter]) { + val databaseName = hiveTable.getDbName + val tableName = hiveTable.getTableName + val tblProps = hiveTable.getParameters + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + val memoryTable = getOrCreateMemoryTable( + databaseName, + tableName, + CacheType.fromString(hiveTable.getProperty("shark.cache")), + preferredStorageLevel, + defaultDiskSerDe = hiveTable.getDeserializer.getClass.getName, + tblProps) val tableSchema = hiveTable.getSchema val serDe = Class.forName(memoryTable.diskSerDe).newInstance.asInstanceOf[Deserializer] serDe.initialize(conf, tableSchema) @@ -199,65 +200,69 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } - def createPartitionedTable( + def getOrCreatePartitionedTable( databaseName: String, tableName: String, + cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, defaultDiskSerDe: String, tblProps: JavaMap[String, String], partSpecs: JavaMap[String, String]): PartitionedMemoryTable = { - val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) - val newPartitionedTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( - databaseName, - tableName, - cacheMode, - preferredStorageLevel, - unifyView = true, - tblProps) - newPartitionedTable.diskSerDe = defaultDiskSerDe - HiveUtils.alterSerdeInHive( - tableName, - Some(partSpecs), - classOf[ColumnarSerDe].getName, - conf) - newPartitionedTable - } - - def loadPartitionedTable( - hiveTable: HiveTable, - partSpecs: JavaMap[String, String], - hadoopReader: HadoopTableReader, - pathFilter: Option[PathFilter]) { - // TODO(harvey): Multiple partition specs... - val databaseName = hiveTable.getDbName - val tableName = hiveTable.getTableName - val tblProps = hiveTable.getParameters - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) - val partitionedTable = work.commandType match { + work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { - createPartitionedTable( + val newPartitionedTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( databaseName, tableName, + cacheMode, preferredStorageLevel, - hiveTable.getDeserializer.getClass.getName, - tblProps, - partSpecs) + unifyView = true, + tblProps) + newPartitionedTable.diskSerDe = defaultDiskSerDe + HiveUtils.alterSerdeInHive( + tableName, + Some(partSpecs), + classOf[ColumnarSerDe].getName, + conf) + newPartitionedTable } case _ => { SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { case Some(table: PartitionedMemoryTable) => table case _ => { - throw new Exception("Invalid state: cached table being updated doesn't exist.") + throw new Exception( + "Internal error: cached, partitioned table for INSERT handling doesn't exist.") } } } } + } + + def loadPartitionedTable( + hiveTable: HiveTable, + partSpecs: JavaMap[String, String], + hadoopReader: HadoopTableReader, + pathFilter: Option[PathFilter]) { + // TODO(harvey): Multiple partition specs... + val databaseName = hiveTable.getDbName + val tableName = hiveTable.getTableName + val tblProps = hiveTable.getParameters + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + val cacheMode = CacheType.fromString(hiveTable.getProperty("shark.cache")) + val partitionedTable = getOrCreatePartitionedTable( + databaseName, + tableName, + cacheMode, + preferredStorageLevel, + defaultDiskSerDe = hiveTable.getDeserializer.getClass.getName, + tblProps, + partSpecs) val partCols = hiveTable.getPartCols.map(_.getName) val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpecs) val partition = db.getPartition(hiveTable, partSpecs, false /* forceCreate */) - val partSerDe = Class.forName(partitionedTable.getDiskSerDe(partitionKey).getOrElse( - partitionedTable.diskSerDe)).newInstance.asInstanceOf[Deserializer] + val partSerDeName = partitionedTable.getDiskSerDe(partitionKey). + getOrElse(partitionedTable.diskSerDe) + val partSerDe = Class.forName(partSerDeName).newInstance.asInstanceOf[Deserializer] val partSchema = partition.getSchema partSerDe.initialize(conf, partSchema) val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) @@ -269,32 +274,35 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe preferredStorageLevel, hadoopReader.broadcastedHiveConf, unionOI) - work.commandType match { - case SparkLoadWork.CommandTypes.OVERWRITE | SparkLoadWork.CommandTypes.NEW_ENTRY => { - partitionedTable.putPartition(partitionKey, tablePartitionRDD) - } - case SparkLoadWork.CommandTypes.INSERT => { - val previousRDD = partitionedTable.getPartition(partitionKey) match { - case Some(previousRDD) => { + val addNewPartitionEntry = partitionedTable.getPartition(partitionKey) match { + case Some(previousRDD) => { + work.commandType match { + case SparkLoadWork.CommandTypes.NEW_ENTRY | SparkLoadWork.CommandTypes.OVERWRITE => true + case SparkLoadWork.CommandTypes.INSERT => { partitionedTable.updatePartition( - partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) + partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) // Note: these matches have to be separate, since an empty partition is represented by // an empty RDD. If it's already cached in memory, then // PartitionedMemoryTable#updatePartition() must be called. // Union stats for the previous RDD with the new RDD loaded. - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName ) match { + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) case None => Unit } + false } - case None => partitionedTable.putPartition(partitionKey, tablePartitionRDD) } } + case None => true + } + if (addNewPartitionEntry) { + partitionedTable.putPartition(partitionKey, tablePartitionRDD) + // If a new partition is added, then the table's SerDe should be used by default. + partitionedTable.setDiskSerDe(partitionKey, partitionedTable.diskSerDe) } SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } - def unionStatsMaps( targetStatsMap: ArrayBuffer[(Int, TablePartitionStats)], otherStatsMap: Iterable[(Int, TablePartitionStats)] From 7434f0751656e7f764a589e420e6fd81cd650645 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 8 Nov 2013 23:42:29 -0800 Subject: [PATCH 229/331] Some logging for MemoryMetadataManager for resting unified table SerDes during shutdown. --- .../memstore2/MemoryMetadataManager.scala | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 36c5c570..837f9966 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -28,13 +28,14 @@ import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.storage.StorageLevel -import shark.execution.RDDUtils +import shark.LogHelper import shark.SharkConfVars import shark.SharkEnv +import shark.execution.RDDUtils import shark.util.HiveUtils -class MemoryMetadataManager { +class MemoryMetadataManager extends LogHelper { private val _keyToTable: ConcurrentMap[String, Table] = new ConcurrentHashMap[String, Table]() @@ -175,21 +176,25 @@ class MemoryMetadataManager { def resetUnifiedTableSerdes() { for (table <- _keyToTable.values.filter(_.unifyView)) { + val tableName = table.tableName + val diskSerDe = table.diskSerDe + // Reset unified table SerDes, so that next time we start up in Shark, those tables can + // be read from disk without special handling. + logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) + HiveUtils.alterSerdeInHive( + tableName, + None /* partitionSpecOpt */, + diskSerDe) table match { - case memoryTable: MemoryTable => { - // Reset unified table SerDes, so that next time we start up in Shark, those tables can - // be read from disk without special handling. - HiveUtils.alterSerdeInHive( - memoryTable.tableName, - None /* partitionSpecOpt */, - memoryTable.diskSerDe) - } case partitionedTable: PartitionedMemoryTable => { for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { + logInfo("Setting SerDe for table %s(partition %s) back to %s.". + format(tableName, hiveKeyStr, serDeName)) val partitionSpec = MemoryMetadataManager.parseHivePartitionKeyStr(hiveKeyStr) HiveUtils.alterSerdeInHive(partitionedTable.tableName, Some(partitionSpec), serDeName) } } + case memoryTable: MemoryTable => Unit } } } @@ -197,7 +202,6 @@ class MemoryMetadataManager { private def makeTableKey(databaseName: String, tableName: String): String = { (databaseName + '.' + tableName).toLowerCase } - } From 1841f5ace3480f19e67a8f9297e395c6cc5954f0 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 9 Nov 2013 03:48:32 -0800 Subject: [PATCH 230/331] Some more cleanup in load and semantic analyzers. --- .../parse/SharkLoadSemanticAnalyzer.scala | 39 +++++++++---------- .../shark/parse/SharkSemanticAnalyzer.scala | 20 +++++----- 2 files changed, 27 insertions(+), 32 deletions(-) diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index 40800032..2443f250 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -47,26 +47,23 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con // Find the arguments needed to instantiate a SparkLoadWork. val tableSpec = new BaseSemanticAnalyzer.tableSpec(db, conf, tableASTNode) val hiveTable = tableSpec.tableHandle - val partSpecOpt = Option(tableSpec.getPartSpec()) - val dataPath = if (partSpecOpt.isEmpty) { - // Non-partitioned table. - hiveTable.getPath - } else { - // Partitioned table. - val partition = db.getPartition(hiveTable, partSpecOpt.get, false /* forceCreate */) - partition.getPartitionPath - } val moveTask = getMoveTask() - val loadCommandType = if (moveTask.getWork.getLoadTableWork.getReplace()) { - SparkLoadWork.CommandTypes.OVERWRITE - } else { - SparkLoadWork.CommandTypes.INSERT - } - - // Capture a snapshot of the data directory being read. When executed, SparkLoadTask will - // determine the input paths to read using a filter that only accepts files not included in - // snapshot set (i.e., the accepted file is a new one created by the Hive load process). - val fileFilter = Utils.createSnapshotFilter(dataPath, conf) + val partSpec = tableSpec.getPartSpec + val (loadCommandType, pathFilterOpt) = + if (moveTask.getWork.getLoadTableWork.getReplace) { + (SparkLoadWork.CommandTypes.OVERWRITE, None) + } else { + val pathOpt = if (hiveTable.isPartitioned) { + Option(db.getPartition(hiveTable, partSpec, false /* forceCreate */)). + map(_.getPartitionPath) + } else { + Some(hiveTable.getPath) + } + // Capture a snapshot of the data directory being read. When executed, SparkLoadTask will + // determine the input paths to read using a filter that only accepts files not included + // in snapshot set (i.e., the accepted file's a new one created by the Hive load process). + (SparkLoadWork.CommandTypes.INSERT, pathOpt.map(Utils.createSnapshotFilter(_, conf))) + } // Create a SparkLoadTask that will use a HadoopRDD to read from the source directory. Set it // to be a dependent task of the LoadTask so that the SparkLoadTask is executed only if the @@ -74,9 +71,9 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con val sparkLoadWork = new SparkLoadWork( databaseName, tableName, - partSpecOpt, + partSpec, loadCommandType, - Some(fileFilter)) + pathFilterOpt) moveTask.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index c1ee6016..73c16293 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -382,9 +382,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with new SparkLoadWork( qb.createTableDesc.getDatabaseName, qb.createTableDesc.getTableName, - partSpecOpt = None, + None /* partSpecsOpt */, SparkLoadWork.CommandTypes.NEW_ENTRY, - pathFilter = None) + None /* pathFilterOpt */) } else { // Split from 'databaseName.tableName' val tableNameSplit = qb.targetTableDesc.getTableName.split('.') @@ -393,14 +393,12 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val hiveTable = db.getTable(databaseName, cachedTableName) val destPartition = qb.getMetaData.getDestPartitionForAlias( qb.getParseInfo.getClauseNamesForDest.head) - val partitionSpec = if (destPartition == null) null else destPartition.getSpec - val isInsertInto = qb.getParseInfo.isInsertIntoTable(cachedTableName) - val (insertType, pathFilterOpt) = - if (isInsertInto) { + val partSpec = if (destPartition == null) null else destPartition.getSpec + val (loadType, pathFilterOpt) = + if (qb.getParseInfo.isInsertIntoTable(cachedTableName)) { val pathOpt = if (hiveTable.isPartitioned) { - val partitionOpt = Option( - db.getPartition(hiveTable, partitionSpec, false /* forceCreate */)) - partitionOpt.map(_.getPartitionPath) + Option(db.getPartition(hiveTable, partSpec, false /* forceCreate */)). + map(_.getPartitionPath) } else { Some(hiveTable.getPath) } @@ -415,8 +413,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with new SparkLoadWork( databaseName, cachedTableName, - Option(partitionSpec), - insertType, + partSpec, + loadType, pathFilterOpt) } val sparkLoadTask = TaskFactory.get(sparkLoadWork, conf) From 4d574dd5758a1e32da63b4bfb4404f935df45bf1 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 9 Nov 2013 05:02:42 -0800 Subject: [PATCH 231/331] Support handling altered "shark.cache" properties. --- .../scala/shark/execution/SharkDDLTask.scala | 3 +- .../scala/shark/execution/SparkLoadTask.scala | 170 +++++++++--------- .../memstore2/MemoryMetadataManager.scala | 6 +- .../parse/SharkDDLSemanticAnalyzer.scala | 97 ++++++++-- 4 files changed, 165 insertions(+), 111 deletions(-) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 0264aeb5..3331793c 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -77,7 +77,8 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] /** Handles a CREATE TABLE or CTAS. */ def createTable( hiveMetadataDb: Hive, - createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { + createTblDesc: CreateTableDesc, + cacheMode: CacheType.CacheType) { val dbName = hiveMetadataDb.getCurrentDatabase() val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index e47a9976..466a3a76 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -51,10 +51,21 @@ private[shark] class SparkLoadWork( val databaseName: String, val tableName: String, - val partSpecOpt: Option[JavaMap[String, String]], + val partSpecsOpt: Option[Seq[JavaMap[String, String]]], val commandType: SparkLoadWork.CommandTypes.Type, - val pathFilter: Option[PathFilter]) - extends java.io.Serializable + val pathFilterOpt: Option[PathFilter], + val unifyView: Boolean = true) + extends java.io.Serializable { + + def this( + databaseName: String, + tableName: String, + partSpec: JavaMap[String, String], + commandType: SparkLoadWork.CommandTypes.Type, + pathFilterOpt: Option[PathFilter]) { + this(databaseName, tableName, Option(partSpec).map(Seq(_)), commandType, pathFilterOpt) + } +} object SparkLoadWork { object CommandTypes extends Enumeration { @@ -76,19 +87,19 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val hadoopReader = new HadoopTableReader(Utilities.getTableDesc(hiveTable), conf) - work.partSpecOpt match { - case Some(partSpec) => { + work.partSpecsOpt match { + case Some(partSpecs) => { loadPartitionedTable( hiveTable, - partSpec, + partSpecs, hadoopReader, - work.pathFilter) + work.pathFilterOpt) } case None => { loadMemoryTable( hiveTable, hadoopReader, - work.pathFilter) + work.pathFilterOpt) } } // Success! @@ -126,18 +137,20 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe (transformedRdd, statsAcc.value) } - def getOrCreateMemoryTable( - databaseName: String, - tableName: String, - cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel, - defaultDiskSerDe: String, - tblProps: JavaMap[String, String]): MemoryTable = { + def getOrCreateMemoryTable(hiveTable: HiveTable): MemoryTable = { + val databaseName = hiveTable.getDbName + val tableName = hiveTable.getTableName + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + hiveTable.getProperty("shark.cache.storageLevel")) work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { val newMemoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, cacheMode, preferredStorageLevel, unifyView = true) - newMemoryTable.diskSerDe = defaultDiskSerDe + databaseName, + tableName, + CacheType.fromString(hiveTable.getProperty("shark.cache")), + preferredStorageLevel, + hiveTable.getProperty("shark.cache.unifyView").toBoolean) + newMemoryTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( tableName, partitionSpecOpt = None, @@ -159,30 +172,21 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe def loadMemoryTable( hiveTable: HiveTable, hadoopReader: HadoopTableReader, - pathFilter: Option[PathFilter]) { + pathFilterOpt: Option[PathFilter]) { val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName - val tblProps = hiveTable.getParameters - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) - val memoryTable = getOrCreateMemoryTable( - databaseName, - tableName, - CacheType.fromString(hiveTable.getProperty("shark.cache")), - preferredStorageLevel, - defaultDiskSerDe = hiveTable.getDeserializer.getClass.getName, - tblProps) + val memoryTable = getOrCreateMemoryTable(hiveTable) val tableSchema = hiveTable.getSchema val serDe = Class.forName(memoryTable.diskSerDe).newInstance.asInstanceOf[Deserializer] serDe.initialize(conf, tableSchema) val inputRDD = hadoopReader.makeRDDForTable( hiveTable, - pathFilter, + pathFilterOpt, serDe.getClass) val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( inputRDD, tableSchema, - preferredStorageLevel, + memoryTable.preferredStorageLevel, hadoopReader.broadcastedHiveConf, serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) memoryTable.tableRDD = work.commandType match { @@ -201,23 +205,22 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } def getOrCreatePartitionedTable( - databaseName: String, - tableName: String, - cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel, - defaultDiskSerDe: String, - tblProps: JavaMap[String, String], + hiveTable: HiveTable, partSpecs: JavaMap[String, String]): PartitionedMemoryTable = { + val databaseName = hiveTable.getDbName + val tableName = hiveTable.getTableName + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + hiveTable.getProperty("shark.cache.storageLevel")) work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { val newPartitionedTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( databaseName, tableName, - cacheMode, + CacheType.fromString(hiveTable.getProperty("shark.cache")), preferredStorageLevel, - unifyView = true, - tblProps) - newPartitionedTable.diskSerDe = defaultDiskSerDe + hiveTable.getProperty("shark.cache.unifyView").toBoolean, + hiveTable.getParameters) + newPartitionedTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( tableName, Some(partSpecs), @@ -239,9 +242,9 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe def loadPartitionedTable( hiveTable: HiveTable, - partSpecs: JavaMap[String, String], + partSpecs: Seq[JavaMap[String, String]], hadoopReader: HadoopTableReader, - pathFilter: Option[PathFilter]) { + pathFilterOpt: Option[PathFilter]) { // TODO(harvey): Multiple partition specs... val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName @@ -249,58 +252,45 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( tblProps.get("shark.cache.storageLevel")) val cacheMode = CacheType.fromString(hiveTable.getProperty("shark.cache")) - val partitionedTable = getOrCreatePartitionedTable( - databaseName, - tableName, - cacheMode, - preferredStorageLevel, - defaultDiskSerDe = hiveTable.getDeserializer.getClass.getName, - tblProps, - partSpecs) val partCols = hiveTable.getPartCols.map(_.getName) - val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpecs) - val partition = db.getPartition(hiveTable, partSpecs, false /* forceCreate */) - val partSerDeName = partitionedTable.getDiskSerDe(partitionKey). - getOrElse(partitionedTable.diskSerDe) - val partSerDe = Class.forName(partSerDeName).newInstance.asInstanceOf[Deserializer] - val partSchema = partition.getSchema - partSerDe.initialize(conf, partSchema) - val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) - val inputRDD = hadoopReader.makeRDDForPartitionedTable( - Map(partition -> partSerDe.getClass), pathFilter) - val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( - inputRDD, - addPartitionInfoToSerDeProps(partCols, new Properties(partition.getSchema)), - preferredStorageLevel, - hadoopReader.broadcastedHiveConf, - unionOI) - val addNewPartitionEntry = partitionedTable.getPartition(partitionKey) match { - case Some(previousRDD) => { - work.commandType match { - case SparkLoadWork.CommandTypes.NEW_ENTRY | SparkLoadWork.CommandTypes.OVERWRITE => true - case SparkLoadWork.CommandTypes.INSERT => { - partitionedTable.updatePartition( - partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) - // Note: these matches have to be separate, since an empty partition is represented by - // an empty RDD. If it's already cached in memory, then - // PartitionedMemoryTable#updatePartition() must be called. - // Union stats for the previous RDD with the new RDD loaded. - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { - case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) - case None => Unit - } - false - } + for (partSpec <- partSpecs) { + val partitionedTable = getOrCreatePartitionedTable(hiveTable, partSpec) + val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) + val partition = db.getPartition(hiveTable, partSpec, false /* forceCreate */) + val partSerDeName = partitionedTable.getDiskSerDe(partitionKey). + getOrElse(partitionedTable.diskSerDe) + val partSerDe = Class.forName(partSerDeName).newInstance.asInstanceOf[Deserializer] + val partSchema = partition.getSchema + partSerDe.initialize(conf, partSchema) + val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) + val inputRDD = hadoopReader.makeRDDForPartitionedTable( + Map(partition -> partSerDe.getClass), pathFilterOpt) + val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + inputRDD, + addPartitionInfoToSerDeProps(partCols, new Properties(partition.getSchema)), + preferredStorageLevel, + hadoopReader.broadcastedHiveConf, + unionOI) + val tableOpt = partitionedTable.getPartition(partitionKey) + if (tableOpt.isDefined && (work.commandType == SparkLoadWork.CommandTypes.INSERT)) { + val previousRDD = tableOpt.get + partitionedTable.updatePartition( + partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) + // Note: these matches have to be separate, since an empty partition is represented by + // an empty RDD. If it's already cached in memory, then + // PartitionedMemoryTable#updatePartition() must be called. + // Union stats for the previous RDD with the new RDD loaded. + SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { + case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) + case None => Unit } + } else { + partitionedTable.putPartition(partitionKey, tablePartitionRDD) + // If a new partition is added, then the table's SerDe should be used by default. + partitionedTable.setDiskSerDe(partitionKey, partitionedTable.diskSerDe) } - case None => true - } - if (addNewPartitionEntry) { - partitionedTable.putPartition(partitionKey, tablePartitionRDD) - // If a new partition is added, then the table's SerDe should be used by default. - partitionedTable.setDiskSerDe(partitionKey, partitionedTable.diskSerDe) + SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } def unionStatsMaps( diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 837f9966..1d0f6589 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -174,12 +174,14 @@ class MemoryMetadataManager extends LogHelper { resetUnifiedTableSerdes() } + /** + * Resets SerDe properties for unified tables to the ones used for deserializing reads. + * That way, tables can be read from disk when the Shark session restarts. + */ def resetUnifiedTableSerdes() { for (table <- _keyToTable.values.filter(_.unifyView)) { val tableName = table.tableName val diskSerDe = table.diskSerDe - // Reset unified table SerDes, so that next time we start up in Shark, those tables can - // be read from disk without special handling. logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) HiveUtils.alterSerdeInHive( tableName, diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 8b1e5625..6985be5a 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -17,6 +17,8 @@ package shark.parse +import java.util.{HashMap => JavaHashMap} + import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf @@ -25,13 +27,13 @@ import org.apache.hadoop.hive.ql.parse.ASTNode import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer import org.apache.hadoop.hive.ql.parse.HiveParser -import org.apache.hadoop.hive.ql.plan.DDLWork +import org.apache.hadoop.hive.ql.plan.{AlterTableDesc, DDLWork} import org.apache.spark.rdd.{UnionRDD, RDD} -import shark.execution.SharkDDLWork +import shark.execution.{SharkDDLWork, SparkLoadWork} import shark.{LogHelper, SharkEnv} -import shark.memstore2.MemoryMetadataManager +import shark.memstore2.{CacheType, MemoryMetadataManager} class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) with LogHelper { @@ -40,22 +42,79 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) super.analyzeInternal(ast) ast.getToken.getType match { - case HiveParser.TOK_DROPTABLE => { - analyzeDropTableOrDropParts(ast) + case HiveParser.TOK_ALTERTABLE_ADDPARTS => { + analyzeAlterTableAddParts(ast) } case HiveParser.TOK_ALTERTABLE_DROPPARTS => { analyzeDropTableOrDropParts(ast) } - case HiveParser.TOK_ALTERTABLE_ADDPARTS => { - analyzeAlterTableAddParts(ast) - } case HiveParser.TOK_ALTERTABLE_RENAME => { analyzeAlterTableRename(ast) } + case HiveParser.TOK_ALTERTABLE_PROPERTIES => { + analyzeAlterTableProperties(ast) + } + case HiveParser.TOK_DROPTABLE => { + analyzeDropTableOrDropParts(ast) + } case _ => Unit } } + /** + * Handle table property changes. + * How Shark-specific changes are handled: + * - "shark.cache": + * If 'true', then create a SparkLoadTask to load the Hive table into memory. + * Set it as a dependent of the Hive DDLTask. A SharkDDLTask counterpart isn't created because + * the HadoopRDD creation and transformation isn't a direct Shark metastore operation + * (unlike the other cases handled in SharkDDLSemantiAnalyzer). + * + * TODO(harvey): Add "uncache" handling. + * If 'false', then create a SharkDDLTask that will delete the table entry in the Shark + * metastore. + * - "shark.cache.unifyView" : + * If 'true' and "shark.cache" is true, then the SparkLoadTask created should read this from the + * table properties when adding an entry to the Shark metastore. + * - "shark.cache.storageLevel": + * Throw an exception since we can't change the storage level without rescanning the entire RDD. + * + * TODO(harvey): Add this, though reevaluate it too...some Spark RDDs might depend on the old + * version of the RDD, so simply dropping it might not work. + */ + def analyzeAlterTableProperties(ast: ASTNode) { + val databaseName = db.getCurrentDatabase() + val tableName = getTableName(ast) + val hiveTable = db.getTable(databaseName, tableName) + val newTblProps = getAlterTblDesc().getProps + val oldTblProps = hiveTable.getParameters + + val isAlreadyCached = CacheType.fromString(oldTblProps.get("shark.cache")) == CacheType.HEAP + val shouldCache = CacheType.fromString(newTblProps.get("shark.cache")) == CacheType.HEAP + if (!isAlreadyCached && shouldCache) { + val partSpecsOpt = + if (SharkEnv.memoryMetadataManager.isHivePartitioned(databaseName, tableName)) { + val columnNames = hiveTable.getPartCols.map(_.getName) + val partSpecs = db.getPartitions(hiveTable).map { partition => + val partSpec = new JavaHashMap[String, String]() + val values = partition.getValues() + columnNames.zipWithIndex.map { case(name, index) => partSpec.put(name, values(index)) } + partSpec + } + Some(partSpecs.toSeq) + } else { + None + } + val sparkLoadTask = new SparkLoadWork( + databaseName, + tableName, + partSpecsOpt, + SparkLoadWork.CommandTypes.NEW_ENTRY, + None /* pathFilterOpt */) + rootTasks.head.addDependentTask(TaskFactory.get(sparkLoadTask, conf)) + } + } + def analyzeDropTableOrDropParts(ast: ASTNode) { val databaseName = db.getCurrentDatabase() val tableName = getTableName(ast) @@ -92,20 +151,22 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) if (SharkEnv.memoryMetadataManager.containsTable(databaseName, oldTableName)) { val newTableName = BaseSemanticAnalyzer.getUnescapedName( astNode.getChild(1).asInstanceOf[ASTNode]) - - // Hive's DDLSemanticAnalyzer#AnalyzeInternal() will only populate rootTasks with a DDLTask - // and DDLWork that contains an AlterTableDesc. - assert(rootTasks.size == 1) - val ddlTask = rootTasks.head - val ddlWork = ddlTask.getWork - assert(ddlWork.isInstanceOf[DDLWork]) - - val alterTableDesc = ddlWork.asInstanceOf[DDLWork].getAlterTblDesc + val alterTableDesc = getAlterTblDesc() val sharkDDLWork = new SharkDDLWork(alterTableDesc) - ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) + rootTasks.head.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } + private def getAlterTblDesc(): AlterTableDesc = { + // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with a DDLTask + // and DDLWork that contains an AlterTableDesc. + assert(rootTasks.size == 1) + val ddlTask = rootTasks.head + val ddlWork = ddlTask.getWork + assert(ddlWork.isInstanceOf[DDLWork]) + ddlWork.asInstanceOf[DDLWork].getAlterTblDesc + } + private def getTableName(node: ASTNode): String = { BaseSemanticAnalyzer.getUnescapedName(node.getChild(0).asInstanceOf[ASTNode]) } From b7e0a7aaa4827afad7fbe3f1ad765468c1969b41 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 9 Nov 2013 14:26:21 -0800 Subject: [PATCH 232/331] Rewrite CACHE to ALTER TABLE --- src/main/scala/shark/SharkDriver.scala | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 1718a51c..c747c9f5 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -111,6 +111,8 @@ private[shark] object SharkDriver extends LogHelper { def getOp = this.op def getCmd() = this.cmd } + + val CACHE_KEYWORD = "CACHE" } @@ -158,6 +160,12 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe } } + def rewriteCacheCmd(cmd: String): String = { + val tableName = cmd.toUpperCase.stripPrefix(SharkDriver.CACHE_KEYWORD) + "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache.unifyView' = 'true', 'shark.cache' = 'true')". + format(tableName) + } + /** * Overload compile to use Shark's semantic analyzers. */ @@ -179,9 +187,16 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe saveSession(queryState) try { - val command = new VariableSubstitution().substitute(conf, cmd) + val command = { + val varSubbedCmd = new VariableSubstitution().substitute(conf, cmd) + if (varSubbedCmd.toUpperCase.startsWith(SharkDriver.CACHE_KEYWORD)) { + rewriteCacheCmd(varSubbedCmd) + } else { + varSubbedCmd + } + } context = new QueryContext(conf, useTableRddSink) - context.setCmd(cmd) + context.setCmd(command) context.setTryCount(getTryCount()) val tree = ParseUtils.findRootNonNullToken((new ParseDriver()).parse(command, context)) From 93bf6c15accb86a78230a16c633e4e270de651bc Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 9 Nov 2013 17:58:23 -0800 Subject: [PATCH 233/331] Slightly nicer parsing of CACHE in SharkDriver. --- src/main/scala/shark/SharkDriver.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index c747c9f5..72c5d2c8 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -161,9 +161,14 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe } def rewriteCacheCmd(cmd: String): String = { - val tableName = cmd.toUpperCase.stripPrefix(SharkDriver.CACHE_KEYWORD) - "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache.unifyView' = 'true', 'shark.cache' = 'true')". - format(tableName) + val cmdSplit = cmd.split(' ') + if (cmdSplit.size == 2) { + val tableName = cmdSplit(1) + "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache.unifyView' = 'true', 'shark.cache' = 'true')". + format(tableName) + } else { + throw new SemanticException("CACHE accepts a single table name: 'CACHE '") + } } /** From 947094072093a5d6586957f0227260567f208d9f Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 9 Nov 2013 23:18:29 -0800 Subject: [PATCH 234/331] Few patches: - Add 'databaseName' entry to Shark Table - Be consistent in the 'conf' used for resetUnifiedTableSerDes() - Don't use Shark metastore to determine partitioned tables in SharkDDLSemanticAnalyzer. --- .../scala/shark/execution/SharkDDLTask.scala | 1 + .../scala/shark/execution/SparkLoadTask.scala | 2 ++ .../memstore2/MemoryMetadataManager.scala | 27 ++++++++++++++----- .../scala/shark/memstore2/MemoryTable.scala | 3 ++- .../memstore2/PartitionedMemoryTable.scala | 8 ++++-- src/main/scala/shark/memstore2/Table.scala | 1 + .../parse/SharkDDLSemanticAnalyzer.scala | 23 ++++++++-------- src/main/scala/shark/util/HiveUtils.scala | 8 +++--- 8 files changed, 49 insertions(+), 24 deletions(-) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 3331793c..11d9b9d0 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -112,6 +112,7 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] // The alternatives are to either attach a HiveDDLTask dependent to this SharkDDLTask or to // copy the necessary code from Hive. HiveUtils.alterSerdeInHive( + dbName, tableName, partitionSpecOpt = None, classOf[ColumnarSerDe].getName, diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 466a3a76..b734e39a 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -152,6 +152,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe hiveTable.getProperty("shark.cache.unifyView").toBoolean) newMemoryTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( + databaseName, tableName, partitionSpecOpt = None, classOf[ColumnarSerDe].getName, @@ -222,6 +223,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe hiveTable.getParameters) newPartitionedTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( + databaseName, tableName, Some(partSpecs), classOf[ColumnarSerDe].getName, diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 1d0f6589..69c189e4 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -82,7 +82,12 @@ class MemoryMetadataManager extends LogHelper { unifyView: Boolean ): MemoryTable = { val tableKey = makeTableKey(databaseName, tableName) - var newTable = new MemoryTable(tableKey, cacheMode, preferredStorageLevel, unifyView) + var newTable = new MemoryTable( + databaseName, + tableName, + cacheMode, + preferredStorageLevel, + unifyView) _keyToTable.put(tableKey, newTable) return newTable } @@ -97,7 +102,8 @@ class MemoryMetadataManager extends LogHelper { ): PartitionedMemoryTable = { val tableKey = makeTableKey(databaseName, tableName) var newTable = new PartitionedMemoryTable( - tableKey, + databaseName, + tableName, cacheMode, preferredStorageLevel, unifyView) @@ -180,20 +186,29 @@ class MemoryMetadataManager extends LogHelper { */ def resetUnifiedTableSerdes() { for (table <- _keyToTable.values.filter(_.unifyView)) { + val conf = Hive.get().getConf val tableName = table.tableName + val databaseName = table.databaseName val diskSerDe = table.diskSerDe - logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) + logError("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) HiveUtils.alterSerdeInHive( + databaseName, tableName, None /* partitionSpecOpt */, - diskSerDe) + diskSerDe, + conf) table match { case partitionedTable: PartitionedMemoryTable => { for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { - logInfo("Setting SerDe for table %s(partition %s) back to %s.". + logError("Setting SerDe for table %s(partition %s) back to %s.". format(tableName, hiveKeyStr, serDeName)) val partitionSpec = MemoryMetadataManager.parseHivePartitionKeyStr(hiveKeyStr) - HiveUtils.alterSerdeInHive(partitionedTable.tableName, Some(partitionSpec), serDeName) + HiveUtils.alterSerdeInHive( + databaseName, + tableName, + Some(partitionSpec), + serDeName, + conf) } } case memoryTable: MemoryTable => Unit diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 25577a32..2c9e82e1 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -28,11 +28,12 @@ import org.apache.spark.storage.StorageLevel */ private[shark] class MemoryTable( + databaseName: String, tableName: String, cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, unifiedView: Boolean) - extends Table(tableName, cacheMode, preferredStorageLevel, unifiedView) { + extends Table(databaseName, tableName, cacheMode, preferredStorageLevel, unifiedView) { // RDD that contains the contents of this table. private var _tableRDD: RDD[TablePartition] = _ diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 49e82284..565509ac 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -37,11 +37,12 @@ import org.apache.spark.storage.StorageLevel */ private[shark] class PartitionedMemoryTable( + databaseName: String, tableName: String, cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, unifiedView: Boolean) - extends Table(tableName, cacheMode, preferredStorageLevel, unifiedView) { + extends Table(databaseName, tableName, cacheMode, preferredStorageLevel, unifiedView) { /** * A simple, mutable wrapper for an RDD. This is needed so that a entry maintained by a @@ -105,7 +106,10 @@ class PartitionedMemoryTable( def removePartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) - if (rddRemoved.isDefined) _cachePolicy.notifyRemove(partitionKey) + _keyToDiskSerDes.remove(partitionKey) + if (rddRemoved.isDefined) { + _cachePolicy.notifyRemove(partitionKey) + } return rddRemoved.map(_.rdd) } diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 63fe9ece..5a7af368 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -32,6 +32,7 @@ import org.apache.spark.storage.StorageLevel * Hive-partition RDD eviction policy. */ private[shark] abstract class Table( + var databaseName: String, var tableName: String, var cacheMode: CacheType.CacheType, var preferredStorageLevel: StorageLevel, diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 6985be5a..b29c4d50 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -92,19 +92,18 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) val isAlreadyCached = CacheType.fromString(oldTblProps.get("shark.cache")) == CacheType.HEAP val shouldCache = CacheType.fromString(newTblProps.get("shark.cache")) == CacheType.HEAP if (!isAlreadyCached && shouldCache) { - val partSpecsOpt = - if (SharkEnv.memoryMetadataManager.isHivePartitioned(databaseName, tableName)) { - val columnNames = hiveTable.getPartCols.map(_.getName) - val partSpecs = db.getPartitions(hiveTable).map { partition => - val partSpec = new JavaHashMap[String, String]() - val values = partition.getValues() - columnNames.zipWithIndex.map { case(name, index) => partSpec.put(name, values(index)) } - partSpec - } - Some(partSpecs.toSeq) - } else { - None + val partSpecsOpt = if (hiveTable.isPartitioned) { + val columnNames = hiveTable.getPartCols.map(_.getName) + val partSpecs = db.getPartitions(hiveTable).map { partition => + val partSpec = new JavaHashMap[String, String]() + val values = partition.getValues() + columnNames.zipWithIndex.map { case(name, index) => partSpec.put(name, values(index)) } + partSpec } + Some(partSpecs.toSeq) + } else { + None + } val sparkLoadTask = new SparkLoadWork( databaseName, tableName, diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 2c38bdc5..8d64dea6 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -26,15 +26,16 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.UnionStructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} +import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan.AlterTableDesc import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} @@ -42,7 +43,6 @@ import shark.SharkContext import shark.api.{DataType, DataTypes} - private[shark] object HiveUtils { def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = { @@ -138,6 +138,7 @@ private[shark] object HiveUtils { * @hiveConf Configuration associated with the current SessionState. */ def alterSerdeInHive( + databaseName: String, tableName: String, partitionSpecOpt: Option[JavaMap[String, String]], serDeName: String, @@ -151,6 +152,7 @@ private[shark] object HiveUtils { alterTableDesc.setOldName(tableName) alterTableDesc.setSerdeName(serDeName) alterTableDesc.setPartSpec(partitionSpec) + val db = Hive.get(hiveConf).setCurrentDatabase(databaseName) // Execute the SerDe change against the Hive metastore. val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], From a7d374c8c1f7b430be5e027735472ec45fb31f24 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 9 Nov 2013 15:14:12 -0800 Subject: [PATCH 235/331] Unit tests for unified views --- .../memstore2/MemoryMetadataManager.scala | 4 +- src/test/scala/shark/SQLSuite.scala | 322 +++++++++++++++++- 2 files changed, 306 insertions(+), 20 deletions(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 69c189e4..cf122025 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -177,14 +177,14 @@ class MemoryMetadataManager extends LogHelper { } def shutdown() { - resetUnifiedTableSerdes() + resetUnifiedTableSerDes() } /** * Resets SerDe properties for unified tables to the ones used for deserializing reads. * That way, tables can be read from disk when the Shark session restarts. */ - def resetUnifiedTableSerdes() { + def resetUnifiedTableSerDes() { for (table <- _keyToTable.values.filter(_.unifyView)) { val conf = Hive.get().getConf val tableName = table.tableName diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 7fffab92..291b95a1 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -19,16 +19,18 @@ package shark import java.util.{HashMap => JavaHashMap} +import scala.collection.JavaConversions._ + import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.ql.metadata.Hive - import org.apache.spark.rdd.UnionRDD import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException -import shark.memstore2.PartitionedMemoryTable +import shark.memstore2.{ColumnarSerDe, MemoryMetadataManager, PartitionedMemoryTable} import shark.util.HiveUtils @@ -38,21 +40,16 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val METASTORE_PATH = TestUtils.getMetastorePath() val MASTER = "local" val DEFAULT_DB_NAME = "default" + val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" var sc: SharkContext = _ + var sharkMetastore: MemoryMetadataManager = _ - override def beforeAll() { - sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) - - sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + - METASTORE_PATH + ";create=true") - sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) - - // second db - sc.sql("create database if not exists seconddb") - + /** + * Tables accessible by any test in SQLSuite. Their properties should remain constant across + * tests. + */ + def loadGlobalTables() { // test sc.runSql("drop table if exists test") sc.runSql("CREATE TABLE test (key INT, val STRING)") @@ -95,6 +92,23 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("CREATE TABLE test1_cached AS SELECT * FROM test1") } + override def beforeAll() { + sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) + + sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + + METASTORE_PATH + ";create=true") + sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) + + sharkMetastore = SharkEnv.memoryMetadataManager + + // second db + sc.sql("create database if not exists seconddb") + + loadGlobalTables() + } + override def afterAll() { sc.stop() System.clearProperty("spark.driver.port") @@ -148,6 +162,34 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { unionRDD.rdds.find(_.isInstanceOf[UnionRDD[_]]).isEmpty } + // Takes a sum over the table's 'key' column, for both the cached table and it's copy on disk. + def expectUnifiedKVTable( + cachedTableName: String, + partSpecOpt: Option[Map[String, String]] = None) { + // Check that the table is in memory and is a unified view. + val sharkTableOpt = sharkMetastore.getTable(DEFAULT_DB_NAME, cachedTableName) + assert(sharkTableOpt.isDefined, "Table %s cannot be found in the Shark meatstore") + assert(sharkTableOpt.get.unifyView, "'unifyView' field for table %s is false") + + // Load a non-cached copy of the table into memory. + // Compare 'key' counts. + val cacheSum = sc.sql("select sum(key) from %s".format(cachedTableName))(0) + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, cachedTableName) + val location = partSpecOpt match { + case Some(partSpec) => { + val partition = Hive.get().getPartition(hiveTable, partSpec, false /* forceCreate */) + partition.getDataLocation.toString + } + case None => hiveTable.getDataLocation.toString + } + val diskTableName = "%s_disk_copy".format(cachedTableName) + sc.sql("drop table if exists %s".format(diskTableName)) + sc.sql("create table %s (key int, value string)".format(diskTableName)) + sc.sql("load data local inpath '%s' into table %s".format(location, diskTableName)) + val diskSum = sc.sql("select sum(key) from %s".format(diskTableName))(0) + assert(diskSum == cacheSum, "Sum of keys from cached and disk contents differ") + } + ////////////////////////////////////////////////////////////////////////////// // basic SQL ////////////////////////////////////////////////////////////////////////////// @@ -887,11 +929,21 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe. - HiveUtils.alterSerdeInHive(tableName, None /* partitionSpecOpt */, columnarSerDeName, hiveConf) + HiveUtils.alterSerdeInHive( + DEFAULT_DB_NAME, + tableName, + None /* partitionSpecOpt */, + columnarSerDeName, + hiveConf) assert(getTableSerDeName(tableName) == columnarSerDeName) // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive(tableName, None /* partitionSpecOpt */, oldSerDeName, hiveConf) + HiveUtils.alterSerdeInHive( + DEFAULT_DB_NAME, + tableName, + None /* partitionSpecOpt */, + oldSerDeName, + hiveConf) assert(getTableSerDeName(tableName) == oldSerDeName) } @@ -915,11 +967,245 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe - HiveUtils.alterSerdeInHive(tableName, Some(partitionSpec), columnarSerDeName, hiveConf) + HiveUtils.alterSerdeInHive( + DEFAULT_DB_NAME, + tableName, + Some(partitionSpec), + columnarSerDeName, + hiveConf) assert(getPartitionSerDeName(tableName, partitionSpec) == columnarSerDeName) // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive(tableName, Some(partitionSpec), oldSerDeName, hiveConf) + HiveUtils.alterSerdeInHive( + DEFAULT_DB_NAME, + tableName, + Some(partitionSpec), + oldSerDeName, + hiveConf) assert(getPartitionSerDeName(tableName, partitionSpec) == oldSerDeName) } + + + ////////////////////////////////////////////////////////////////////////////// + // Creating unified views + ////////////////////////////////////////////////////////////////////////////// + test ("Cached table created by CREATE TABLE, with table properties, is unified view by default") { + sc.runSql("drop table if exists test_unify_creation") + sc.runSql("""create table test_unify_creation (key int, val string) + tblproperties('shark.cache'='true')""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation").get + assert(table.unifyView) + sc.runSql("drop table if exists test_unify_creation") + } + + test ("Cached table created by CREATE TABLE, with '_cached', is unified view by default") { + sc.runSql("drop table if exists test_unify_creation_cached") + sc.runSql("create table test_unify_creation_cached(key int, val string)") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation_cached").get + assert(table.unifyView) + sc.runSql("drop table if exists test_unify_creation_cached") + } + + test ("Cached table created by CTAS, with table properties, is unified view by default") { + sc.runSql("drop table if exists test_unify_ctas") + sc.runSql("""create table test_unify_ctas + tblproperties('shark.cache' = 'true') as select * from test""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas").get + assert(table.unifyView) + expectSql("select count(*) from test_unify_ctas", "500") + sc.runSql("drop table if exists test_unify_ctas") + } + + test ("Cached table created by CTAS, with '_cached', is unified view by default") { + sc.runSql("drop table if exists test_unify_ctas_cached") + sc.runSql("create table test_unify_ctas_cached as select * from test") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas_cached").get + assert(table.unifyView) + expectSql("select count(*) from test_unify_ctas_cached", "500") + sc.runSql("drop table if exists test_unify_ctas_cached") + } + + test ("Don't maintain unified view for CREATE TABLE when 'shark.cache.unifyView' is false") { + sc.runSql("drop table if exists test_non_unify_creation") + sc.runSql("""create table test_non_unify_creation(key int, val string) + tblproperties('shark.cache' = 'true', 'shark.cache.unifyView' = 'false')""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_creation").get + assert(!table.unifyView) + sc.runSql("drop table if exists test_non_unify_creation") + } + + test ("Don't maintiain unified view for CTAS when 'shark.cache.unifyView' is false") { + sc.runSql("drop table if exists test_non_unify_ctas") + sc.runSql("""create table test_non_unify_ctas tblproperties + ('shark.cache' = 'true', 'shark.cache.unifyView' = 'false') as select * from test""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_ctas").get + assert(!table.unifyView) + sc.runSql("drop table if exists test_non_unify_ctas") + } + + ////////////////////////////////////////////////////////////////////////////// + // LOAD for unified views + ////////////////////////////////////////////////////////////////////////////// + test ("LOAD INTO unified view") { + sc.runSql("drop table if exists unified_view_cached") + sc.runSql("create table unified_view_cached (key int, value string)") + sc.runSql("load data local inpath '%s' into table unified_view_cached".format(KV1_TXT_PATH)) + expectUnifiedKVTable("unified_view_cached") + expectSql("select count(*) from unified_view_cached", "500") + sc.runSql("drop table if exists unified_view_cached") + } + + test ("LOAD OVERWRITE unified view") { + sc.runSql("drop table if exists unified_overwrite_cached") + sc.runSql("create table unified_overwrite_cached (key int, value string)") + sc.runSql("load data local inpath '%s' into table unified_overwrite_cached". + format("${hiveconf:shark.test.data.path}/kv3.txt")) + expectSql("select count(*) from unified_overwrite_cached", "25") + sc.runSql("load data local inpath '%s' overwrite into table unified_overwrite_cached". + format(KV1_TXT_PATH)) + // Make sure the cached contents matches the disk contents. + expectUnifiedKVTable("unified_overwrite_cached") + expectSql("select count(*) from unified_overwrite_cached", "500") + sc.runSql("drop table if exists unified_overwrite_cached") + } + + test ("LOAD INTO partitioned unified view") { + sc.runSql("drop table if exists unified_view_part_cached") + sc.runSql("""create table unified_view_part_cached (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""load data local inpath '%s' into table unified_view_part_cached + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectUnifiedKVTable("unified_view_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_view_part_cached", "500") + sc.runSql("drop table if exists unified_view_part_cached") + } + + test ("LOAD OVERWRITE partitioned unified view") { + sc.runSql("drop table if exists unified_overwrite_part_cached") + sc.runSql("""create table unified_overwrite_part_cached (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""load data local inpath '%s' overwrite into table unified_overwrite_part_cached + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectUnifiedKVTable("unified_overwrite_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_overwrite_part_cached", "500") + sc.runSql("drop table if exists unified_overwrite_part_cached") + } + + ////////////////////////////////////////////////////////////////////////////// + // INSERTS for unified views + ////////////////////////////////////////////////////////////////////////////// + test ("INSERT INTO unified view") { + sc.runSql("drop table if exists unified_view_cached") + sc.runSql("create table unified_view_cached as select * from test_cached") + sc.runSql("insert into table unified_view_cached select * from test_cached") + expectUnifiedKVTable("unified_view_cached") + expectSql("select count(*) from unified_view_cached", "1000") + sc.runSql("drop table if exists unified_view_cached") + } + + test ("INSERT OVERWRITE unified view") { + sc.runSql("drop table if exists unified_overwrite_cached") + sc.runSql("create table unified_overwrite_cached as select * from test") + sc.runSql("insert overwrite table unified_overwrite_cached select * from test_cached") + expectUnifiedKVTable("unified_overwrite_cached") + expectSql("select count(*) from unified_overwrite_cached", "500") + sc.runSql("drop table if exists unified_overwrite_cached") + } + + test ("INSERT INTO partitioned unified view") { + sc.runSql("drop table if exists unified_view_part_cached") + sc.runSql("""create table unified_view_part_cached (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""insert into table unified_view_part_cached partition (keypart = 1) + select * from test_cached""") + expectUnifiedKVTable("unified_view_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_view_part_cached where keypart = 1", "500") + sc.runSql("drop table if exists unified_view_part_cached") + } + + test ("INSERT OVERWRITE partitioned unified view") { + sc.runSql("drop table if exists unified_overwrite_part_cached") + sc.runSql("""create table unified_overwrite_part_cached (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""insert overwrite table unified_overwrite_part_cached partition (keypart = 1) + select * from test_cached""") + expectUnifiedKVTable("unified_overwrite_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_overwrite_part_cached", "500") + sc.runSql("drop table if exists unified_overwrite_part_cached") + } + + ////////////////////////////////////////////////////////////////////////////// + // CACHE and ALTER TABLE commands + ////////////////////////////////////////////////////////////////////////////// + test ("ALTER TABLE caches contents of non-partitioned table if 'shark.cache' is set to true") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("""alter table unified_load set + tblproperties('shark.cache' = 'true', 'shark.cache.unifyView' = 'true')""") + expectUnifiedKVTable("unified_load") + sc.runSql("drop table if exists unified_load") + } + + test ("ALTER TABLE caches all contents of partitioned table if 'shark.cache' is set to true") { + sc.runSql("drop table if exists unified_part_load") + sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") + sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") + sc.runSql("""alter table unified_part_load set + tblproperties('shark.cache' = 'true', 'shark.cache.unifyView' = 'true')""") + expectUnifiedKVTable("unified_part_load", Some(Map("keypart" -> "1"))) + sc.runSql("drop table if exists unified_part_load") + } + + test ("CACHE behaves like ALTER TABLE SET TBLPROPERTIES ...") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("cache unified_load") + // Double check the table properties. + val tableName = "unified_load" + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) + assert(hiveTable.getProperty("shark.cache") == "true") + assert(hiveTable.getProperty("shark.cache.unifyView") == "true") + // Check that the cache and disk contents are synchronized. + expectUnifiedKVTable(tableName) + sc.runSql("drop table if exists unified_load") + } + + ////////////////////////////////////////////////////////////////////////////// + // Unified view persistence + ////////////////////////////////////////////////////////////////////////////// + + test ("Unified views persist across Shark metastore shutdowns.") { + val columnarSerDeName = classOf[ColumnarSerDe].getName + // All cached tables are unified by default, so MemoryMetadataManager#resetUnifiedTableSerDes() + // should reset SerDes for the SQLSuite-global tables. + val globalCachedTableNames = Seq("test_cached", "test_null_cached", "clicks_cached", + "users_cached", "test1_cached") + val db = Hive.get() + val cachedTableCounts = new Array[String](globalCachedTableNames.size) + for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { + val cachedCount = sc.sql("select count(*) from %s".format(tableName))(0) + val cacheSerDe = db.getTable(DEFAULT_DB_NAME, tableName) + .getDeserializer.getClass.getName + + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, tableName).get + assert(table.unifyView) + assert(table.diskSerDe == classOf[LazySimpleSerDe].getName) + assert(cacheSerDe == columnarSerDeName) + cachedTableCounts(i) = cachedCount + } + sharkMetastore.resetUnifiedTableSerDes() + for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { + // Make sure the SerDe has been reset. + val diskSerDe = Hive.get().getTable(DEFAULT_DB_NAME, tableName).getDeserializer.getClass.getName + assert(diskSerDe != columnarSerDeName, """SerDe for %s wasn't reset across Shark metastore + restart. (disk SerDe: %s)""".format(tableName, diskSerDe)) + // Check that the number of rows remain the same + val onDiskCount = sc.sql("select count(*) from %s".format(tableName))(0) + val cachedCount = cachedTableCounts(i) + assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. + (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) + } + // Finally, reload all tables + loadGlobalTables() + } } From 180dddf8ea99b9b3e1c13bb448b9b6ea1ef144d9 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 10 Nov 2013 00:03:09 -0800 Subject: [PATCH 236/331] Add some asserts in Shark Table for unified views, plus some other minor cleanup. --- .../memstore2/PartitionedMemoryTable.scala | 7 ++++++- src/main/scala/shark/memstore2/Table.scala | 7 +++++-- src/test/scala/shark/SQLSuite.scala | 20 ++++++++----------- 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 565509ac..55c731ed 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -59,6 +59,8 @@ class PartitionedMemoryTable( private var _keyToPartitions: ConcurrentMap[String, RDDValue] = new ConcurrentJavaHashMap[String, RDDValue]() + // Map from Hive-partition key to the SerDe name used to deserialize rows read from disk. + // Should only be used for unified views. private var _keyToDiskSerDes: ConcurrentMap[String, String] = new ConcurrentJavaHashMap[String, String]() @@ -129,7 +131,10 @@ class PartitionedMemoryTable( _cachePolicy = newPolicy } - def setDiskSerDe(partitionKey: String, serDe: String) = _keyToDiskSerDes.put(partitionKey, serDe) + def setDiskSerDe(partitionKey: String, serDe: String) = { + assert(unifyView, "Setting diskSerDe for %s, but it isn't a unified view.".format(tableName)) + _keyToDiskSerDes.put(partitionKey, serDe) + } def getDiskSerDe(partitionKey: String): Option[String] = _keyToDiskSerDes.get(partitionKey) diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 5a7af368..c724377d 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -38,10 +38,13 @@ private[shark] abstract class Table( var preferredStorageLevel: StorageLevel, var unifyView: Boolean) { - // SerDe used to read from and write to disk. + // SerDe used to read from and write to disk. Should only be set for unified views. private var _diskSerDe: String = _ def diskSerDe: String = _diskSerDe - def diskSerDe_= (newSerDe: String) = _diskSerDe = newSerDe + def diskSerDe_= (newSerDe: String) = { + assert(unifyView, "Setting diskSerDe for %s, but it isn't a unified view.".format(tableName)) + _diskSerDe = newSerDe + } } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 291b95a1..59be0e2c 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -162,7 +162,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { unionRDD.rdds.find(_.isInstanceOf[UnionRDD[_]]).isEmpty } - // Takes a sum over the table's 'key' column, for both the cached table and it's copy on disk. + // Takes a sum over the table's 'key' column, for both the cached contents and the copy on disk. def expectUnifiedKVTable( cachedTableName: String, partSpecOpt: Option[Map[String, String]] = None) { @@ -172,7 +172,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(sharkTableOpt.get.unifyView, "'unifyView' field for table %s is false") // Load a non-cached copy of the table into memory. - // Compare 'key' counts. val cacheSum = sc.sql("select sum(key) from %s".format(cachedTableName))(0) val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, cachedTableName) val location = partSpecOpt match { @@ -182,6 +181,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } case None => hiveTable.getDataLocation.toString } + // Create a table with contents loaded from the table's data directory. val diskTableName = "%s_disk_copy".format(cachedTableName) sc.sql("drop table if exists %s".format(diskTableName)) sc.sql("create table %s (key int, value string)".format(diskTableName)) @@ -1111,7 +1111,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { expectSql("select count(*) from unified_overwrite_cached", "500") sc.runSql("drop table if exists unified_overwrite_cached") } - +a test ("INSERT INTO partitioned unified view") { sc.runSql("drop table if exists unified_view_part_cached") sc.runSql("""create table unified_view_part_cached (key int, value string) @@ -1173,39 +1173,35 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // Unified view persistence ////////////////////////////////////////////////////////////////////////////// - test ("Unified views persist across Shark metastore shutdowns.") { val columnarSerDeName = classOf[ColumnarSerDe].getName // All cached tables are unified by default, so MemoryMetadataManager#resetUnifiedTableSerDes() // should reset SerDes for the SQLSuite-global tables. val globalCachedTableNames = Seq("test_cached", "test_null_cached", "clicks_cached", "users_cached", "test1_cached") - val db = Hive.get() + // Number of rows for each cached table. val cachedTableCounts = new Array[String](globalCachedTableNames.size) for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { val cachedCount = sc.sql("select count(*) from %s".format(tableName))(0) - val cacheSerDe = db.getTable(DEFAULT_DB_NAME, tableName) + val cacheSerDe = Hive.get().getTable(DEFAULT_DB_NAME, tableName) .getDeserializer.getClass.getName - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, tableName).get - assert(table.unifyView) - assert(table.diskSerDe == classOf[LazySimpleSerDe].getName) assert(cacheSerDe == columnarSerDeName) cachedTableCounts(i) = cachedCount } sharkMetastore.resetUnifiedTableSerDes() for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { - // Make sure the SerDe has been reset. + // Make sure the SerDe has been reset to the one used for deserializing disk reads. val diskSerDe = Hive.get().getTable(DEFAULT_DB_NAME, tableName).getDeserializer.getClass.getName assert(diskSerDe != columnarSerDeName, """SerDe for %s wasn't reset across Shark metastore restart. (disk SerDe: %s)""".format(tableName, diskSerDe)) - // Check that the number of rows remain the same + // Check that the number of rows from the table on disk remains the same. val onDiskCount = sc.sql("select count(*) from %s".format(tableName))(0) val cachedCount = cachedTableCounts(i) assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) } - // Finally, reload all tables + // Finally, reload all tables. loadGlobalTables() } } From 20513378e0c6f2a258484043e9378189a08b1879 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 10 Nov 2013 01:24:06 -0800 Subject: [PATCH 237/331] Cleanup and more comments. --- src/main/scala/shark/SharkDriver.scala | 4 + src/main/scala/shark/Utils.scala | 2 +- .../shark/execution/OperatorFactory.scala | 4 + .../scala/shark/execution/SharkDDLTask.scala | 6 +- .../scala/shark/execution/SparkLoadTask.scala | 158 ++++++++++++++---- .../shark/execution/TableScanOperator.scala | 2 +- .../memstore2/MemoryMetadataManager.scala | 8 +- src/main/scala/shark/parse/QueryBlock.scala | 11 +- .../parse/SharkLoadSemanticAnalyzer.scala | 26 +-- .../shark/parse/SharkSemanticAnalyzer.scala | 120 +++++++------ src/main/scala/shark/util/HiveUtils.scala | 5 +- src/test/scala/shark/SQLSuite.scala | 2 +- 12 files changed, 219 insertions(+), 129 deletions(-) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 72c5d2c8..0af2acaf 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -160,6 +160,10 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe } } + /** + * Rewrites a CACHE command to + * ALTER TABLE SET TBLPROPERTIES ('shark.cache.unifyView' = 'true', 'shark.cache' = 'true'). + */ def rewriteCacheCmd(cmd: String): String = { val cmdSplit = cmd.split(' ') if (cmdSplit.size == 2) { diff --git a/src/main/scala/shark/Utils.scala b/src/main/scala/shark/Utils.scala index 7bf3a75f..c4c94cfa 100644 --- a/src/main/scala/shark/Utils.scala +++ b/src/main/scala/shark/Utils.scala @@ -95,7 +95,7 @@ object Utils { } /** - * Returns a filter that accepts files missing from the current snapshot of the `path` directory. + * Returns a filter that accepts files not in the current snapshot of the `path` directory. */ def createSnapshotFilter(path: Path, conf: Configuration): PathFilter = { val fs = path.getFileSystem(conf) diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 26ec7742..7139cf1d 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -79,6 +79,10 @@ object OperatorFactory extends LogHelper { _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } + /** + * Make sure that the FileSinkOperator uses the disk SerDe for deserializing input rows. + * For example, it should use a LazySimpleSerDe to deserialize LazySimpleStructObjectInspectors. + */ def createUnifiedViewFileOutputPlan( hiveTerminalOp: HOperator[_<:HiveDesc], diskSerDe: String): TerminalOperator = { diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 11d9b9d0..b927a423 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -107,10 +107,10 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] if (unifyView) { val table = hiveMetadataDb.getTable(tableName) newTable.diskSerDe = table.getDeserializer().getClass.getName - // This creates and directly executes a Hive DDLTask to change the table's SerDe property in + // Creates and directly execute a Hive DDLTask to change the table's SerDe property in // the Hive metastore. - // The alternatives are to either attach a HiveDDLTask dependent to this SharkDDLTask or to - // copy the necessary code from Hive. + // The alternatives are to either attach a HiveDDLTask dependent on this SharkDDLTask or to + // copy the necessary code from Hive, both of which are more troublesome... HiveUtils.alterSerdeInHive( dbName, tableName, diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index b734e39a..d8195028 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -36,12 +36,12 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObj import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat} +import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.SerializableWritable import org.apache.spark.storage.StorageLevel -import shark.{LogHelper, SharkEnv} +import shark.{LogHelper, SharkEnv, Utils} import shark.execution.serialization.KryoSerializer import shark.memstore2._ import shark.util.HiveUtils @@ -60,10 +60,10 @@ class SparkLoadWork( def this( databaseName: String, tableName: String, - partSpec: JavaMap[String, String], + partSpecOpt: Option[JavaMap[String, String]], commandType: SparkLoadWork.CommandTypes.Type, pathFilterOpt: Option[PathFilter]) { - this(databaseName, tableName, Option(partSpec).map(Seq(_)), commandType, pathFilterOpt) + this(databaseName, tableName, partSpecOpt.map(Seq(_)), commandType, pathFilterOpt) } } @@ -72,8 +72,49 @@ object SparkLoadWork { type Type = Value val OVERWRITE, INSERT, NEW_ENTRY = Value } + + /** + * Factory/helper method used in LOAD and INSERT INTO/OVERWRITE analysis. + * + * A path filter is created if the command is an INSERT and under these conditions: + * - Table is partitioned, and the partition being updated already exists + * (i.e., `partSpecOpt.isDefined == true`) + * - Table is not partitioned - Hive is used to check for whether it exists. + */ + def apply( + db: Hive, + conf: HiveConf, + hiveTable: HiveTable, + partSpecOpt: Option[JavaMap[String, String]], + isOverwrite: Boolean): SparkLoadWork = { + val (loadType, pathFilterOpt) = + if (isOverwrite) { + (SparkLoadWork.CommandTypes.OVERWRITE, None) + } else { + val pathFilterOpt = if (hiveTable.isPartitioned) { + partSpecOpt.flatMap { partSpec => + // Partition being updated exists + val partitionOpt = Option(db.getPartition(hiveTable, partSpec, false /* forceCreate */)) + partitionOpt.map(part => Utils.createSnapshotFilter(part.getPartitionPath, conf)) + } + } else { + Some(Utils.createSnapshotFilter(hiveTable.getPath, conf)) + } + (SparkLoadWork.CommandTypes.INSERT, pathFilterOpt) + } + new SparkLoadWork( + hiveTable.getDbName, + hiveTable.getTableName, + partSpecOpt, + loadType, + pathFilterOpt) + } } +/** + * A Hive task to load data from disk into the Shark cache. Handles INSERT INTO/OVERWRITE, + * LOAD INTO/OVERWRITE, CACHE, and CTAS commands. + */ private[shark] class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHelper { @@ -83,10 +124,8 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val databaseName = work.databaseName val tableName = work.tableName val hiveTable = Hive.get(conf).getTable(databaseName, tableName) - val oi = hiveTable.getDeserializer().getObjectInspector().asInstanceOf[StructObjectInspector] - + // Used to generate HadoopRDDs. val hadoopReader = new HadoopTableReader(Utilities.getTableDesc(hiveTable), conf) - work.partSpecsOpt match { case Some(partSpecs) => { loadPartitionedTable( @@ -106,21 +145,32 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe 0 } + /** + * Returns a materialized, in-memory RDD comprising TablePartitions backed by columnar stores. + * + * @inputRdd A hadoop RDD, or a union of hadoop RDDs if the table is partitioned. + * @serDeProps Properties used to initialize local ColumnarSerDe instantiations. This contains + * the output schema used to create output object inspectors. + * @storageLevel Storage level for the materialized RDD returned. + * @broadcasedHiveConf Allows for sharing a Hive Configruation broadcast used to create the Hadoop + * `inputRdd`. + * @inputOI Object inspector used to read rows from `inputRdd`. + */ def transformAndMaterializeInput( inputRdd: RDD[_], serDeProps: Properties, storageLevel: StorageLevel, broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], - oi: StructObjectInspector) = { + inputOI: StructObjectInspector) = { val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) - val serializedOI = KryoSerializer.serialize(oi) + val serializedOI = KryoSerializer.serialize(inputOI) val transformedRdd = inputRdd.mapPartitionsWithIndex { case (partIndex, partIter) => val serde = new ColumnarSerDe serde.initialize(broadcastedHiveConf.value.value, serDeProps) - val oi = KryoSerializer.deserialize[ObjectInspector](serializedOI) + val localInputOI = KryoSerializer.deserialize[ObjectInspector](serializedOI) var builder: Writable = null partIter.foreach { row => - builder = serde.serialize(row.asInstanceOf[AnyRef], oi) + builder = serde.serialize(row.asInstanceOf[AnyRef], localInputOI) } if (builder == null) { // Empty partition. @@ -132,11 +182,18 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } } transformedRdd.persist(storageLevel) + // Run a job to materialize the RDD at the cache `storageLevel` specified. transformedRdd.context.runJob( transformedRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) (transformedRdd, statsAcc.value) } + /** + * Returns Shark MemoryTable that was created or fetched from the metastore, based on the command + * type handled by this task. + * + * @hiveTable Corresponding HiveTable for which to fetch or create the Shark Memorytable. + */ def getOrCreateMemoryTable(hiveTable: HiveTable): MemoryTable = { val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName @@ -150,6 +207,8 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe CacheType.fromString(hiveTable.getProperty("shark.cache")), preferredStorageLevel, hiveTable.getProperty("shark.cache.unifyView").toBoolean) + // Before setting the table's SerDe property to ColumnarSerDe, record the SerDe used + // to deserialize rows from disk so that it can be used for successive update operations. newMemoryTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( databaseName, @@ -170,6 +229,16 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } } + /** + * Handles loading data from disk into the Shark cache for non-partitioned tables. + * + * @hiveTable Hive metadata object representing the target table. + * @hadoopReader Used to create a HadoopRDD from the table's data directory. + * @pathFilterOpt Defined for INSERT update operations (e.g., INSERT INTO) and passed to + * hadoopReader#makeRDDForTable() to determine which new files should be read from the table's + * data directory - see the SparkLoadWork#apply() factory method for an example of how the + * path filter is created. + */ def loadMemoryTable( hiveTable: HiveTable, hadoopReader: HadoopTableReader, @@ -180,10 +249,12 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val tableSchema = hiveTable.getSchema val serDe = Class.forName(memoryTable.diskSerDe).newInstance.asInstanceOf[Deserializer] serDe.initialize(conf, tableSchema) + // Scan the Hive table's data directory. val inputRDD = hadoopReader.makeRDDForTable( hiveTable, pathFilterOpt, serDe.getClass) + // Transform the HadoopRDD to an RDD[TablePartition]. val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( inputRDD, tableSchema, @@ -194,6 +265,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe case (SparkLoadWork.CommandTypes.OVERWRITE | SparkLoadWork.CommandTypes.NEW_ENTRY) => tablePartitionRDD case SparkLoadWork.CommandTypes.INSERT => { + // Union the previous and new RDDs, and their respective table stats. val unionedRDD = RDDUtils.unionAndFlatten(tablePartitionRDD, memoryTable.tableRDD) SharkEnv.memoryMetadataManager.getStats(databaseName, tableName ) match { case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) @@ -205,6 +277,12 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } + /** + * Returns Shark PartitionedMemorytable that was created or fetched from the metastore, based on + * the command type handled by this task. + * + * @hiveTable Corresponding HiveTable for the returned Shark PartitionedMemorytable. + */ def getOrCreatePartitionedTable( hiveTable: HiveTable, partSpecs: JavaMap[String, String]): PartitionedMemoryTable = { @@ -242,12 +320,25 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } } + /** + * Handles loading data from disk into the Shark cache for non-partitioned tables. + * + * @hiveTable Hive metadata object representing the target table. + * @partSpecs Sequence of partition key specifications that contains either a single key, + * or all of the table's partition keys. This is because only one partition specficiation is + * allowed for each append or overwrite command, and new cache entries (i.e, for a CACHE + * comand) are full table scans. + * @hadoopReader Used to create a HadoopRDD from each partition's data directory. + * @pathFilterOpt Defined for INSERT update operations (e.g., INSERT INTO) and passed to + * hadoopReader#makeRDDForTable() to determine which new files should be read from the table + * partition's data directory - see the SparkLoadWork#apply() factory method for an example of + * how a path filter is created. + */ def loadPartitionedTable( hiveTable: HiveTable, partSpecs: Seq[JavaMap[String, String]], hadoopReader: HadoopTableReader, pathFilterOpt: Option[PathFilter]) { - // TODO(harvey): Multiple partition specs... val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName val tblProps = hiveTable.getParameters @@ -255,32 +346,38 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe tblProps.get("shark.cache.storageLevel")) val cacheMode = CacheType.fromString(hiveTable.getProperty("shark.cache")) val partCols = hiveTable.getPartCols.map(_.getName) + for (partSpec <- partSpecs) { + // Read, materialize, and store a columnar-backed RDD for `partSpec`. val partitionedTable = getOrCreatePartitionedTable(hiveTable, partSpec) val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) val partition = db.getPartition(hiveTable, partSpec, false /* forceCreate */) + + // Name of the SerDe used to deserialize the partition contents on disk. If the partition + // specified doesn't currently exist, then default to the table's disk SerDe. val partSerDeName = partitionedTable.getDiskSerDe(partitionKey). getOrElse(partitionedTable.diskSerDe) val partSerDe = Class.forName(partSerDeName).newInstance.asInstanceOf[Deserializer] val partSchema = partition.getSchema partSerDe.initialize(conf, partSchema) + // Get a UnionStructObjectInspector that unifies the two StructObjectInspectors for the table + // columns and the partition columns. val unionOI = HiveUtils.makeUnionOIForPartitionedTable(partSchema, partSerDe) + // Create a HadoopRDD for the file scan. val inputRDD = hadoopReader.makeRDDForPartitionedTable( Map(partition -> partSerDe.getClass), pathFilterOpt) val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( inputRDD, - addPartitionInfoToSerDeProps(partCols, new Properties(partition.getSchema)), + addPartitionInfoToSerDeProps(partCols, partition.getSchema), preferredStorageLevel, hadoopReader.broadcastedHiveConf, unionOI) + // Determine how to cache the table RDD created. val tableOpt = partitionedTable.getPartition(partitionKey) if (tableOpt.isDefined && (work.commandType == SparkLoadWork.CommandTypes.INSERT)) { val previousRDD = tableOpt.get partitionedTable.updatePartition( partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) - // Note: these matches have to be separate, since an empty partition is represented by - // an empty RDD. If it's already cached in memory, then - // PartitionedMemoryTable#updatePartition() must be called. // Union stats for the previous RDD with the new RDD loaded. SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) @@ -295,20 +392,14 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } } - def unionStatsMaps( - targetStatsMap: ArrayBuffer[(Int, TablePartitionStats)], - otherStatsMap: Iterable[(Int, TablePartitionStats)] - ): ArrayBuffer[(Int, TablePartitionStats)] = { - val targetStatsMapSize = targetStatsMap.size - for ((otherIndex, tableStats) <- otherStatsMap) { - targetStatsMap.append((otherIndex + targetStatsMapSize, tableStats)) - } - targetStatsMap - } - + /** + * Returns a copy of `baseSerDeProps` with row metadata that contains the names and types for the + * table's partitioning columns. + */ def addPartitionInfoToSerDeProps( partCols: Seq[String], - serDeProps: Properties): Properties = { + baseSerDeProps: Properties): Properties = { + val serDeProps = new Properties(baseSerDeProps) // Delimited by "," var columnNameProperty: String = serDeProps.getProperty(Constants.LIST_COLUMNS) // NULL if column types are missing. By default, the SerDeParameters initialized by the @@ -329,6 +420,17 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe serDeProps } + def unionStatsMaps( + targetStatsMap: ArrayBuffer[(Int, TablePartitionStats)], + otherStatsMap: Iterable[(Int, TablePartitionStats)] + ): ArrayBuffer[(Int, TablePartitionStats)] = { + val targetStatsMapSize = targetStatsMap.size + for ((otherIndex, tableStats) <- otherStatsMap) { + targetStatsMap.append((otherIndex + targetStatsMapSize, tableStats)) + } + targetStatsMap + } + override def getType = StageType.MAPRED override def getName = "MAPRED-LOAD-SPARK" diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 05a93be5..1a156de2 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -126,7 +126,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { // Stats used for map pruning. val indexToStatsOpt: Option[collection.Map[Int, TablePartitionStats]] = SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) - assert (indexToStatsOpt.isDefined, "Stats not found for table " + tableName) + assert(indexToStatsOpt.isDefined, "Stats not found for table " + tableName) val indexToStats = indexToStatsOpt.get // Run map pruning if the flag is set, there exists a filter predicate on diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index cf122025..d1a82283 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -190,7 +190,7 @@ class MemoryMetadataManager extends LogHelper { val tableName = table.tableName val databaseName = table.databaseName val diskSerDe = table.diskSerDe - logError("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) + logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) HiveUtils.alterSerdeInHive( databaseName, tableName, @@ -200,7 +200,7 @@ class MemoryMetadataManager extends LogHelper { table match { case partitionedTable: PartitionedMemoryTable => { for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { - logError("Setting SerDe for table %s(partition %s) back to %s.". + logInfo("Setting SerDe for table %s(partition %s) back to %s.". format(tableName, hiveKeyStr, serDeName)) val partitionSpec = MemoryMetadataManager.parseHivePartitionKeyStr(hiveKeyStr) HiveUtils.alterSerdeInHive( @@ -252,6 +252,10 @@ object MemoryMetadataManager { return keyStr } + /** + * Returns a (partition column name -> value) mapping by parsing a `keyStr` of the format + * 'col1=value1/col2=value2/.../colN=valueN', created by makeHivePartitionKeyStr() above. + */ def parseHivePartitionKeyStr(keyStr: String): JavaMap[String, String] = { val partitionSpec = new JavaHashMap[String, String]() for (pair <- keyStr.split("/")) { diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 04c94693..6624f37a 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -57,12 +57,13 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) def unifyView: Boolean = _unifyView - def targetTableDesc: TableDesc = _targetTableDesc - - def targetTableDesc_= (desc: TableDesc) = _targetTableDesc = desc - - // Hive uses "tableDesc" to refer to the CreateTableDesc... + // Hive uses "tableDesc" to refer to the CreateTableDesc. This makes it easier to differentiate + // from `_targetTableDesc`. def createTableDesc: CreateTableDesc = super.getTableDesc def createTableDesc_= (desc: CreateTableDesc) = super.setTableDesc(desc) + + def targetTableDesc: TableDesc = _targetTableDesc + + def targetTableDesc_= (desc: TableDesc) = _targetTableDesc = desc } diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index 2443f250..25b582a7 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, LoadSeman import org.apache.hadoop.hive.ql.plan._ import shark.execution.SparkLoadWork import shark.{LogHelper, SharkEnv} -import shark.Utils class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(conf) { @@ -48,32 +47,13 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con val tableSpec = new BaseSemanticAnalyzer.tableSpec(db, conf, tableASTNode) val hiveTable = tableSpec.tableHandle val moveTask = getMoveTask() - val partSpec = tableSpec.getPartSpec - val (loadCommandType, pathFilterOpt) = - if (moveTask.getWork.getLoadTableWork.getReplace) { - (SparkLoadWork.CommandTypes.OVERWRITE, None) - } else { - val pathOpt = if (hiveTable.isPartitioned) { - Option(db.getPartition(hiveTable, partSpec, false /* forceCreate */)). - map(_.getPartitionPath) - } else { - Some(hiveTable.getPath) - } - // Capture a snapshot of the data directory being read. When executed, SparkLoadTask will - // determine the input paths to read using a filter that only accepts files not included - // in snapshot set (i.e., the accepted file's a new one created by the Hive load process). - (SparkLoadWork.CommandTypes.INSERT, pathOpt.map(Utils.createSnapshotFilter(_, conf))) - } + val partSpecOpt = Option(tableSpec.getPartSpec) + val isOverwrite = moveTask.getWork.getLoadTableWork.getReplace + val sparkLoadWork = SparkLoadWork(db, conf, hiveTable, partSpecOpt, isOverwrite) // Create a SparkLoadTask that will use a HadoopRDD to read from the source directory. Set it // to be a dependent task of the LoadTask so that the SparkLoadTask is executed only if the // Hive task executes successfully. - val sparkLoadWork = new SparkLoadWork( - databaseName, - tableName, - partSpec, - loadCommandType, - pathFilterOpt) moveTask.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 73c16293..43d10dcd 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -72,10 +72,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with * For CTAS and INSERT INTO/OVERWRITE the generated Shark query plan matches the one * created if the target table were not cached. Disk => memory loading is done by a * SparkLoadTask that executes _after_ all other tasks (SparkTask, Hive MoveTasks) finish - * executing. For INSERT INTO, this allows the SparkLoadTask to determine, based on a - * snapshot of the table/partition data directory taken in genMapRedTasks(), and load new - * file content into the cache. For CTAS, everything in the data directory is loaded into - * the cache. + * executing. For INSERT INTO, the SparkLoadTask will be able to determine, using a path filter + * based on a snapshot of the table/partition data directory taken in genMapRedTasks(), new files + * that should be loaded into the cache. For CTAS, a path filter isn't used - everything in the + * data directory is loaded into the cache. * * Non-unified views (i.e., the cached table content is memory-only): * The query plan's FileSinkOperator is replaced by a MemoryStoreSinkOperator. The @@ -170,7 +170,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with pCtx.getTopOps().values().head) // TODO: clean the following code. It's too messy to understand... - val terminalOpSeq: Seq[TerminalOperator] = { + val terminalOpSeq = { val qbParseInfo = qb.getParseInfo if (qbParseInfo.isInsertToTable && !qb.isCTAS) { // Handle INSERT. There can be multiple Hive sink operators if the single command comprises @@ -190,7 +190,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with if (hiveSinkOps.size == 1) { val table = SharkEnv.memoryMetadataManager.getTable( databaseName, cachedTableName).get - // INSERT update on a cached table. + // INSERT INTO or OVERWRITE update on a cached table. qb.targetTableDesc = tableDesc // If useUnionRDD is true, the sink op is for INSERT INTO. val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) @@ -208,6 +208,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with new String } if (table.unifyView) { + // The table being updated is a unified view, a SparkLoadTask will be created + // by the genMapRedTasks() call below. Set fields in `qb` that will be needed. qb.unifyView = true qb.targetTableDesc = tableDesc qb.preferredStorageLevel = preferredStorageLevel @@ -240,39 +242,39 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } } } else if (hiveSinkOps.size == 1) { - // For a single output, we have the option of choosing the output - // destination (e.g. CTAS with table property "shark.cache" = "true"). - if (qb.isCTAS && qb.getTableDesc != null && - CacheType.shouldCache(qb.cacheModeForCreateTable)) { - // The table being created from CTAS will be cached. Check whether it should be - // synchronized with disk (i.e., maintain a unified view) or memory-only. - val tblProps = qb.getTableDesc().getTblProps - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) - if (qb.unifyView) { - // Save the preferred storage level - needed to create a SparkLoadTask in - // genMapRedTasks(). - // Create the usual Shark file output plan. - qb.preferredStorageLevel = preferredStorageLevel - Seq(OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head)) + Seq { + // For a single output, we have the option of choosing the output + // destination (e.g. CTAS with table property "shark.cache" = "true"). + if (qb.isCTAS && qb.getTableDesc != null && + CacheType.shouldCache(qb.cacheModeForCreateTable)) { + // The table being created from CTAS should be cached. Check whether it should be + // synchronized with disk (i.e., maintain a unified view) or memory-only. + val tblProps = qb.getTableDesc().getTblProps + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + if (qb.unifyView) { + // Save the preferred storage level, since it's needed to create a SparkLoadTask in + // genMapRedTasks(). + qb.preferredStorageLevel = preferredStorageLevel + OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head) + } else { + qb.getTableDesc().getTblProps().put(CachedTableRecovery.QUERY_STRING, ctx.getCmd()) + OperatorFactory.createSharkMemoryStoreOutputPlan( + hiveSinkOps.head, + qb.getTableDesc.getTableName, + qb.getTableDesc.getDatabaseName, + preferredStorageLevel, + _resSchema.size, /* numColumns */ + new String, /* hivePartitionKey */ + qb.cacheModeForCreateTable, + false /* useUnionRDD */) + } + } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { + OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) } else { - qb.getTableDesc().getTblProps().put(CachedTableRecovery.QUERY_STRING, ctx.getCmd()) - Seq(OperatorFactory.createSharkMemoryStoreOutputPlan( - hiveSinkOps.head, - qb.getTableDesc.getTableName, - qb.getTableDesc.getDatabaseName, - preferredStorageLevel, - _resSchema.size, /* numColumns */ - new String, /* hivePartitionKey */ - qb.cacheModeForCreateTable, - false /* useUnionRDD */)) + OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head) } - } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { - Seq(OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head)) - } else { - Seq(OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head)) } - // A hack for the query plan dashboard to get the query plan. This was // done for SIGMOD demo. Turn it off by default. //shark.dashboard.QueryPlanDashboardHandler.terminalOperator = terminalOp @@ -317,10 +319,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with setFetchTask(fetchTask) } else { - // Configure MoveTasks for table updates (CTAS, INSERT). + // Configure MoveTasks for CTAS, INSERT. val mvTasks = new ArrayList[MoveTask]() - // For CTAS, 'fileWork' contains a single LoadFileDesc (called "LoadFileWork" in Hive). + // For CTAS, `fileWork` contains a single LoadFileDesc (called "LoadFileWork" in Hive). val fileWork = getParseContext.getLoadFileWork val tableWork = getParseContext.getLoadTableWork tableWork.foreach { ltd => @@ -330,7 +332,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with fileWork.foreach { lfd => if (qb.isCTAS) { - // For CTAS, lfd.targetDir should be the data directory of the table being created. + // For CTAS, `lfd.targetDir` references the data directory of the table being created. var location = qb.getTableDesc.getLocation if (location == null) { try { @@ -378,7 +380,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } if (qb.unifyView) { + // Create a SparkLoadTask used to scan and load disk contents into the cache. val sparkLoadWork = if (qb.isCTAS) { + // No need to pass a filter, since the entire table data directory should be loaded, or + // pass partition specifications, since partitioned tables can't be created from CTAS. new SparkLoadWork( qb.createTableDesc.getDatabaseName, qb.createTableDesc.getTableName, @@ -386,37 +391,23 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with SparkLoadWork.CommandTypes.NEW_ENTRY, None /* pathFilterOpt */) } else { + // Determine the path filter to use for an INSERT INTO and fetch the partition key + // specifications, if the table being updated is partitioned. + // Split from 'databaseName.tableName' val tableNameSplit = qb.targetTableDesc.getTableName.split('.') val databaseName = tableNameSplit(0) val cachedTableName = tableNameSplit(1) val hiveTable = db.getTable(databaseName, cachedTableName) - val destPartition = qb.getMetaData.getDestPartitionForAlias( - qb.getParseInfo.getClauseNamesForDest.head) - val partSpec = if (destPartition == null) null else destPartition.getSpec - val (loadType, pathFilterOpt) = - if (qb.getParseInfo.isInsertIntoTable(cachedTableName)) { - val pathOpt = if (hiveTable.isPartitioned) { - Option(db.getPartition(hiveTable, partSpec, false /* forceCreate */)). - map(_.getPartitionPath) - } else { - Some(hiveTable.getPath) - } - (SparkLoadWork.CommandTypes.INSERT, pathOpt.map(Utils.createSnapshotFilter(_, conf))) - } else { - (SparkLoadWork.CommandTypes.OVERWRITE, None) - } - - // Add a SparkLoadTask as a dependent of all MoveTasks, so that when executed, the table's - // (or table partition's) data directory will contain the updates that need to be loaded - // into memory. - new SparkLoadWork( - databaseName, - cachedTableName, - partSpec, - loadType, - pathFilterOpt) + // None if the table isn't partitioned, or if the partition specified doesn't exist. + val partSpecOpt = Option(qb.getMetaData.getDestPartitionForAlias( + qb.getParseInfo.getClauseNamesForDest.head)).map(_.getSpec) + val isOverwrite = !qb.getParseInfo.isInsertIntoTable(cachedTableName) + SparkLoadWork(db, conf, hiveTable, partSpecOpt, isOverwrite) } + // Add a SparkLoadTask as a dependent of all MoveTasks, so that when executed, the table's + // (or table partition's) data directory will already contain updates that should be + // loaded into memory. val sparkLoadTask = TaskFactory.get(sparkLoadWork, conf) mvTasks.foreach(_.addDependentTask(sparkLoadTask)) } @@ -512,6 +503,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean createTableProperties.put("shark.cache.unifyView", queryBlock.unifyView.toString) if (shouldCache && !queryBlock.unifyView) { + // Directly set the ColumnarSerDe if the table persists memory-only. createTableDesc.setSerName(classOf[ColumnarSerDe].getName) } diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 8d64dea6..1b1f96f7 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -61,6 +61,10 @@ private[shark] object HiveUtils { case DataTypes.STRING => PrimitiveObjectInspectorFactory.javaStringObjectInspector } + /** + * Return a UnionStructObjectInspector that combines the StructObjectInspectors for the table + * schema and the partition columns, which are virtual in Hive. + */ def makeUnionOIForPartitionedTable( partProps: Properties, partSerDe: Deserializer): UnionStructObjectInspector = { @@ -165,7 +169,6 @@ private[shark] object HiveUtils { def executeDDLTaskDirectly(ddlWork: DDLWork, hiveConf: HiveConf): Int = { val task = new DDLTask() task.initialize(hiveConf, null /* queryPlan */, null /* ctx: DriverContext */) - task.setWork(ddlWork) // Hive returns 0 if the create table command is executed successfully. diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 59be0e2c..6d1a0f74 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -1111,7 +1111,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { expectSql("select count(*) from unified_overwrite_cached", "500") sc.runSql("drop table if exists unified_overwrite_cached") } -a + test ("INSERT INTO partitioned unified view") { sc.runSql("drop table if exists unified_view_part_cached") sc.runSql("""create table unified_view_part_cached (key int, value string) From e34a896db5d8a8c925a9750ec20afe167447c160 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 10 Nov 2013 12:15:40 -0800 Subject: [PATCH 238/331] Comment for HiveUtils#executeDDLTaskDirectly(). --- src/main/scala/shark/util/HiveUtils.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 1b1f96f7..53fc6806 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -166,12 +166,16 @@ private[shark] object HiveUtils { taskExecutionStatus == 0 } + /** + * Creates a DDLTask from the DDLWork given, and directly calls DDLTask#execute(). Returns 0 if + * the create table command is executed successfully. + * This is safe to use for all DDL commands except for AlterTableTypes.ARCHIVE, which actually + * requires the DriverContext created in Hive Driver#execute(). + */ def executeDDLTaskDirectly(ddlWork: DDLWork, hiveConf: HiveConf): Int = { val task = new DDLTask() task.initialize(hiveConf, null /* queryPlan */, null /* ctx: DriverContext */) task.setWork(ddlWork) - - // Hive returns 0 if the create table command is executed successfully. task.execute(null /* driverContext */) } } From de3dad7e5b12336ccc92ac8a0f52b8d02fe8e5e1 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 10 Nov 2013 13:05:33 -0800 Subject: [PATCH 239/331] Add back missed import after merge --- src/main/scala/shark/memstore2/MemoryMetadataManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index aeac777c..80be5204 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -18,7 +18,7 @@ package shark.memstore2 import java.util.concurrent.ConcurrentHashMap -import java.util.{Map => JavaMap} +import java.util.{HashMap=> JavaHashMap, Map => JavaMap} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, ConcurrentMap} From a946017dc266c55f784f36e0bdb3905dc841edf4 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 10 Nov 2013 14:20:21 -0800 Subject: [PATCH 240/331] Don't set "shark.cache.unifyView" for all tables created in Shark. It might actually be better that Shark table properties (shark.cache.unifyView, shark.cache) are set for non-cached tables, but this interferes with Hive tests that use the output of "describe formatted to compute a diff. --- .../scala/shark/parse/SharkSemanticAnalyzer.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index d193dfc0..240fa2f5 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -496,13 +496,14 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } val shouldCache = CacheType.shouldCache(cacheMode) - queryBlock.unifyView = shouldCache && - createTableProperties.getOrElse("shark.cache.unifyView", - SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean - createTableProperties.put("shark.cache.unifyView", queryBlock.unifyView.toString) - if (shouldCache && !queryBlock.unifyView) { - // Directly set the ColumnarSerDe if the table persists memory-only. - createTableDesc.setSerName(classOf[ColumnarSerDe].getName) + if (shouldCache) { + queryBlock.unifyView = createTableProperties.getOrElse("shark.cache.unifyView", + SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean + createTableProperties.put("shark.cache.unifyView", queryBlock.unifyView.toString) + if (!queryBlock.unifyView) { + // Directly set the ColumnarSerDe if the table will be stored memory-only. + createTableDesc.setSerName(classOf[ColumnarSerDe].getName) + } } // For CTAS ('isRegularCreateTable' is false), the MemoryStoreSinkOperator creates a new From 2c50a8edb259804818c597fe10aa88130f984956 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 10 Nov 2013 17:57:54 -0800 Subject: [PATCH 241/331] Don't read cache mode and preferred storage level within SparkLoadTask. Do it during semantic analysis. --- .../scala/shark/execution/SparkLoadTask.scala | 114 +++++++++--------- .../parse/SharkDDLSemanticAnalyzer.scala | 29 +++-- .../parse/SharkLoadSemanticAnalyzer.scala | 8 +- .../shark/parse/SharkSemanticAnalyzer.scala | 23 ++-- 4 files changed, 94 insertions(+), 80 deletions(-) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index d8195028..31b4d1ae 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -21,7 +21,7 @@ package shark.execution import java.util.{HashMap => JavaHashMap, Properties, Map => JavaMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, Buffer} import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.conf.HiveConf @@ -51,19 +51,19 @@ private[shark] class SparkLoadWork( val databaseName: String, val tableName: String, - val partSpecsOpt: Option[Seq[JavaMap[String, String]]], val commandType: SparkLoadWork.CommandTypes.Type, - val pathFilterOpt: Option[PathFilter], - val unifyView: Boolean = true) + val preferredStorageLevel: StorageLevel, + val cacheMode: CacheType.CacheType, + val unifyView: Boolean) extends java.io.Serializable { - def this( - databaseName: String, - tableName: String, - partSpecOpt: Option[JavaMap[String, String]], - commandType: SparkLoadWork.CommandTypes.Type, - pathFilterOpt: Option[PathFilter]) { - this(databaseName, tableName, partSpecOpt.map(Seq(_)), commandType, pathFilterOpt) + var pathFilterOpt: Option[PathFilter] = None + + var partSpecs: Seq[JavaMap[String, String]] = Nil + + def addPartSpec(partSpec: JavaMap[String, String]) { + // Not the most efficient, but this method isn't called very often. + partSpecs = partSpecs ++ Seq(partSpec) } } @@ -74,7 +74,8 @@ object SparkLoadWork { } /** - * Factory/helper method used in LOAD and INSERT INTO/OVERWRITE analysis. + * Factory/helper method used in LOAD and INSERT INTO/OVERWRITE analysis. This sets all + * necessary fields in SparkLoadWork. * * A path filter is created if the command is an INSERT and under these conditions: * - Table is partitioned, and the partition being updated already exists @@ -86,28 +87,31 @@ object SparkLoadWork { conf: HiveConf, hiveTable: HiveTable, partSpecOpt: Option[JavaMap[String, String]], - isOverwrite: Boolean): SparkLoadWork = { - val (loadType, pathFilterOpt) = - if (isOverwrite) { - (SparkLoadWork.CommandTypes.OVERWRITE, None) - } else { - val pathFilterOpt = if (hiveTable.isPartitioned) { - partSpecOpt.flatMap { partSpec => - // Partition being updated exists - val partitionOpt = Option(db.getPartition(hiveTable, partSpec, false /* forceCreate */)) - partitionOpt.map(part => Utils.createSnapshotFilter(part.getPartitionPath, conf)) - } - } else { - Some(Utils.createSnapshotFilter(hiveTable.getPath, conf)) - } - (SparkLoadWork.CommandTypes.INSERT, pathFilterOpt) - } - new SparkLoadWork( + commandType: SparkLoadWork.CommandTypes.Type): SparkLoadWork = { + val cacheMode = CacheType.fromString(hiveTable.getProperty("shark.cache")) + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + hiveTable.getProperty("shark.cache.storageLevel")) + val sparkLoadWork = new SparkLoadWork( hiveTable.getDbName, hiveTable.getTableName, - partSpecOpt, - loadType, - pathFilterOpt) + commandType, + preferredStorageLevel, + cacheMode, + unifyView = hiveTable.getProperty("shark.cache.unifyView").toBoolean) + partSpecOpt.foreach(sparkLoadWork.addPartSpec(_)) + if (commandType == SparkLoadWork.CommandTypes.INSERT) { + if (hiveTable.isPartitioned) { + partSpecOpt.foreach { partSpec => + // None if the partition being updated doesn't exist yet. + val partitionOpt = Option(db.getPartition(hiveTable, partSpec, false /* forceCreate */)) + sparkLoadWork.pathFilterOpt = partitionOpt.map(part => + Utils.createSnapshotFilter(part.getPartitionPath, conf)) + } + } else { + sparkLoadWork.pathFilterOpt = Some(Utils.createSnapshotFilter(hiveTable.getPath, conf)) + } + } + sparkLoadWork } } @@ -126,20 +130,17 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val hiveTable = Hive.get(conf).getTable(databaseName, tableName) // Used to generate HadoopRDDs. val hadoopReader = new HadoopTableReader(Utilities.getTableDesc(hiveTable), conf) - work.partSpecsOpt match { - case Some(partSpecs) => { - loadPartitionedTable( - hiveTable, - partSpecs, - hadoopReader, - work.pathFilterOpt) - } - case None => { - loadMemoryTable( - hiveTable, - hadoopReader, - work.pathFilterOpt) - } + if (hiveTable.isPartitioned) { + loadPartitionedTable( + hiveTable, + work.partSpecs, + hadoopReader, + work.pathFilterOpt) + } else { + loadMemoryTable( + hiveTable, + hadoopReader, + work.pathFilterOpt) } // Success! 0 @@ -197,16 +198,14 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe def getOrCreateMemoryTable(hiveTable: HiveTable): MemoryTable = { val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - hiveTable.getProperty("shark.cache.storageLevel")) work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { val newMemoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( databaseName, tableName, - CacheType.fromString(hiveTable.getProperty("shark.cache")), - preferredStorageLevel, - hiveTable.getProperty("shark.cache.unifyView").toBoolean) + work.cacheMode, + work.preferredStorageLevel, + work.unifyView) // Before setting the table's SerDe property to ColumnarSerDe, record the SerDe used // to deserialize rows from disk so that it can be used for successive update operations. newMemoryTable.diskSerDe = hiveTable.getDeserializer.getClass.getName @@ -288,16 +287,14 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe partSpecs: JavaMap[String, String]): PartitionedMemoryTable = { val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - hiveTable.getProperty("shark.cache.storageLevel")) work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { val newPartitionedTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( databaseName, tableName, - CacheType.fromString(hiveTable.getProperty("shark.cache")), - preferredStorageLevel, - hiveTable.getProperty("shark.cache.unifyView").toBoolean, + work.cacheMode, + work.preferredStorageLevel, + work.unifyView, hiveTable.getParameters) newPartitionedTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( @@ -342,9 +339,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName val tblProps = hiveTable.getParameters - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) - val cacheMode = CacheType.fromString(hiveTable.getProperty("shark.cache")) val partCols = hiveTable.getPartCols.map(_.getName) for (partSpec <- partSpecs) { @@ -369,7 +363,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( inputRDD, addPartitionInfoToSerDeProps(partCols, partition.getSchema), - preferredStorageLevel, + work.preferredStorageLevel, hadoopReader.broadcastedHiveConf, unionOI) // Determine how to cache the table RDD created. diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index b29c4d50..9b8d858f 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -17,9 +17,10 @@ package shark.parse -import java.util.{HashMap => JavaHashMap} +import java.util.{HashMap => JavaHashMap, Map => JavaMap} import scala.collection.JavaConversions._ +import scala.collection.mutable.Buffer import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.TaskFactory @@ -31,8 +32,8 @@ import org.apache.hadoop.hive.ql.plan.{AlterTableDesc, DDLWork} import org.apache.spark.rdd.{UnionRDD, RDD} +import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.{SharkDDLWork, SparkLoadWork} -import shark.{LogHelper, SharkEnv} import shark.memstore2.{CacheType, MemoryMetadataManager} @@ -89,9 +90,10 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) val newTblProps = getAlterTblDesc().getProps val oldTblProps = hiveTable.getParameters - val isAlreadyCached = CacheType.fromString(oldTblProps.get("shark.cache")) == CacheType.HEAP - val shouldCache = CacheType.fromString(newTblProps.get("shark.cache")) == CacheType.HEAP - if (!isAlreadyCached && shouldCache) { + val oldCacheMode = CacheType.fromString(oldTblProps.get("shark.cache")) + val newCacheMode = CacheType.fromString(newTblProps.get("shark.cache")) + if (!(oldCacheMode == CacheType.HEAP) && (newCacheMode == CacheType.HEAP)) { + // The table should be cached (and is not already cached) val partSpecsOpt = if (hiveTable.isPartitioned) { val columnNames = hiveTable.getPartCols.map(_.getName) val partSpecs = db.getPartitions(hiveTable).map { partition => @@ -100,17 +102,24 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) columnNames.zipWithIndex.map { case(name, index) => partSpec.put(name, values(index)) } partSpec } - Some(partSpecs.toSeq) + Some(partSpecs) } else { None } - val sparkLoadTask = new SparkLoadWork( + val unifyView = newTblProps.getOrElse("shark.cache.unifyView", + SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean + val cacheMode = newCacheMode + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + newTblProps.get("shark.cache.storageLevel")) + val sparkLoadWork = new SparkLoadWork( databaseName, tableName, - partSpecsOpt, SparkLoadWork.CommandTypes.NEW_ENTRY, - None /* pathFilterOpt */) - rootTasks.head.addDependentTask(TaskFactory.get(sparkLoadTask, conf)) + preferredStorageLevel, + cacheMode, + unifyView) + partSpecsOpt.foreach(partSpecs => sparkLoadWork.partSpecs = partSpecs) + rootTasks.head.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } } diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index 25b582a7..2c170ba0 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -48,8 +48,12 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con val hiveTable = tableSpec.tableHandle val moveTask = getMoveTask() val partSpecOpt = Option(tableSpec.getPartSpec) - val isOverwrite = moveTask.getWork.getLoadTableWork.getReplace - val sparkLoadWork = SparkLoadWork(db, conf, hiveTable, partSpecOpt, isOverwrite) + val commandType = if (moveTask.getWork.getLoadTableWork.getReplace) { + SparkLoadWork.CommandTypes.OVERWRITE + } else { + SparkLoadWork.CommandTypes.INSERT + } + val sparkLoadWork = SparkLoadWork(db, conf, hiveTable, partSpecOpt, commandType) // Create a SparkLoadTask that will use a HadoopRDD to read from the source directory. Set it // to be a dependent task of the LoadTask so that the SparkLoadTask is executed only if the diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 240fa2f5..f4312561 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -380,28 +380,35 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with if (qb.unifyView) { // Create a SparkLoadTask used to scan and load disk contents into the cache. val sparkLoadWork = if (qb.isCTAS) { - // No need to pass a filter, since the entire table data directory should be loaded, or + val tblProps = qb.createTableDesc.getTblProps + val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + tblProps.get("shark.cache.storageLevel")) + val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) + + // No need to create a filter, since the entire table data directory should be loaded, nor // pass partition specifications, since partitioned tables can't be created from CTAS. new SparkLoadWork( qb.createTableDesc.getDatabaseName, qb.createTableDesc.getTableName, - None /* partSpecsOpt */, SparkLoadWork.CommandTypes.NEW_ENTRY, - None /* pathFilterOpt */) + preferredStorageLevel, + cacheMode, + unifyView = true) } else { - // Determine the path filter to use for an INSERT INTO and fetch the partition key - // specifications, if the table being updated is partitioned. - // Split from 'databaseName.tableName' val tableNameSplit = qb.targetTableDesc.getTableName.split('.') val databaseName = tableNameSplit(0) val cachedTableName = tableNameSplit(1) + val commandType = if (qb.getParseInfo.isInsertIntoTable(cachedTableName)) { + SparkLoadWork.CommandTypes.INSERT + } else { + SparkLoadWork.CommandTypes.OVERWRITE + } val hiveTable = db.getTable(databaseName, cachedTableName) // None if the table isn't partitioned, or if the partition specified doesn't exist. val partSpecOpt = Option(qb.getMetaData.getDestPartitionForAlias( qb.getParseInfo.getClauseNamesForDest.head)).map(_.getSpec) - val isOverwrite = !qb.getParseInfo.isInsertIntoTable(cachedTableName) - SparkLoadWork(db, conf, hiveTable, partSpecOpt, isOverwrite) + SparkLoadWork(db, conf, hiveTable, partSpecOpt, commandType) } // Add a SparkLoadTask as a dependent of all MoveTasks, so that when executed, the table's // (or table partition's) data directory will already contain updates that should be From efc76e46662671ec0190cee2b7b0bfe0c03461fc Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 10 Nov 2013 19:47:16 -0800 Subject: [PATCH 242/331] Refresh the Hive DB when resetting table properties on shutdown. --- src/main/scala/shark/SharkConfVars.scala | 2 +- .../scala/shark/execution/SparkLoadTask.scala | 1 - .../memstore2/MemoryMetadataManager.scala | 22 ++++++++++++++----- src/test/scala/shark/SQLSuite.scala | 16 ++++++++++---- 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 519310fa..b63e3ad5 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -53,7 +53,7 @@ object SharkConfVars { val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.policy.maxSize", "10") // Default value for the "shark.cache.unify" table property. - val DEFAULT_UNIFY_FLAG = new ConfVar("shark.cache.unify.default", true) + val DEFAULT_UNIFY_FLAG = new ConfVar("shark.cache.unifyView.default", true) // If true, then cache any table whose name ends in "_cached". val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 31b4d1ae..8280d375 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -338,7 +338,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe pathFilterOpt: Option[PathFilter]) { val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName - val tblProps = hiveTable.getParameters val partCols = hiveTable.getPartCols.map(_.getName) for (partSpec <- partSpecs) { diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 80be5204..0e31adad 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -182,11 +182,12 @@ class MemoryMetadataManager extends LogHelper { * That way, tables can be read from disk when the Shark session restarts. */ def resetUnifiedTableSerDes() { - for (table <- _keyToTable.values.filter(_.unifyView)) { - val conf = Hive.get().getConf - val tableName = table.tableName - val databaseName = table.databaseName - val diskSerDe = table.diskSerDe + val db = Hive.get() + for (sharkTable <- _keyToTable.values.filter(_.unifyView)) { + val conf = db.getConf + val tableName = sharkTable.tableName + val databaseName = sharkTable.databaseName + val diskSerDe = sharkTable.diskSerDe logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) HiveUtils.alterSerdeInHive( databaseName, @@ -194,7 +195,16 @@ class MemoryMetadataManager extends LogHelper { None /* partitionSpecOpt */, diskSerDe, conf) - table match { + // Also remove all Shark related table properties from the Hive table metadata. + val hiveTable = db.getTable(databaseName, tableName) + val tblProps = hiveTable.getParameters + tblProps.remove("shark.cache") + tblProps.remove("shark.cache.storageLevel") + tblProps.remove("shark.cache.unifyView") + // Refresh the Hive db. + db.alterTable(tableName, hiveTable) + // Reset SerDes if the table is partitioned. + sharkTable match { case partitionedTable: PartitionedMemoryTable => { for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { logInfo("Setting SerDe for table %s(partition %s) back to %s.". diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 6d1a0f74..0261ef26 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -1182,17 +1182,17 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Number of rows for each cached table. val cachedTableCounts = new Array[String](globalCachedTableNames.size) for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) val cachedCount = sc.sql("select count(*) from %s".format(tableName))(0) - val cacheSerDe = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - .getDeserializer.getClass.getName - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, tableName).get + val cacheSerDe = hiveTable.getDeserializer.getClass.getName assert(cacheSerDe == columnarSerDeName) cachedTableCounts(i) = cachedCount } sharkMetastore.resetUnifiedTableSerDes() for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) // Make sure the SerDe has been reset to the one used for deserializing disk reads. - val diskSerDe = Hive.get().getTable(DEFAULT_DB_NAME, tableName).getDeserializer.getClass.getName + val diskSerDe = hiveTable.getDeserializer.getClass.getName assert(diskSerDe != columnarSerDeName, """SerDe for %s wasn't reset across Shark metastore restart. (disk SerDe: %s)""".format(tableName, diskSerDe)) // Check that the number of rows from the table on disk remains the same. @@ -1200,6 +1200,14 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val cachedCount = cachedTableCounts(i) assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) + // Make sure that Shark table properties are removed. + val tblProps = hiveTable.getParameters + assert(!tblProps.contains("shark.cache"), + "'shark.cache' table property should be removed.") + assert(!tblProps.contains("shark.storageLevel"), + "'shark.storageLevel' table property should be removed.") + assert(!tblProps.contains("shark.unifyView"), + "'shark.unifyView' table property should be removed.") } // Finally, reload all tables. loadGlobalTables() From d200337650c58eac9ca05b0cf741a30bb527b404 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 Nov 2013 00:06:32 -0800 Subject: [PATCH 243/331] Set fair scheduler pool name only if the pool is defined. --- src/main/scala/shark/execution/SparkTask.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index b9e828ec..04dea899 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -91,9 +91,10 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { // Set Spark's job description to be this query. SharkEnv.sc.setJobDescription(work.pctx.getContext.getCmd) - // Set the fair scheduler's pool. - SharkEnv.sc.setLocalProperty("spark.scheduler.pool", - conf.get("mapred.fairscheduler.pool")) + // Set the fair scheduler's pool using mapred.fairscheduler.pool if it is defined. + Option(conf.get("mapred.fairscheduler.pool")).foreach { pool => + SharkEnv.sc.setLocalProperty("spark.scheduler.pool", pool) + } val sinkRdd = terminalOp.execute().asInstanceOf[RDD[Any]] From f295a1d9d9ba50e31c8f118ed257aa3bfd94bf0d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 11 Nov 2013 00:46:54 -0800 Subject: [PATCH 244/331] Set fair scheduler pool name in SparkLoadTask only if the pool is defined (in conjunction with PR 214) --- src/main/scala/shark/SharkConfVars.scala | 2 +- src/main/scala/shark/execution/SparkLoadTask.scala | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index b63e3ad5..519310fa 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -53,7 +53,7 @@ object SharkConfVars { val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.policy.maxSize", "10") // Default value for the "shark.cache.unify" table property. - val DEFAULT_UNIFY_FLAG = new ConfVar("shark.cache.unifyView.default", true) + val DEFAULT_UNIFY_FLAG = new ConfVar("shark.cache.unify.default", true) // If true, then cache any table whose name ends in "_cached". val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 8280d375..5d3e1506 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -125,6 +125,14 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe override def execute(driveContext: DriverContext): Int = { logDebug("Executing " + this.getClass.getName) + // Set Spark's job description to be this query. + SharkEnv.sc.setJobDescription("Loading from Hadoop for a(n) " + work.commandType) + + // Set the fair scheduler's pool using mapred.fairscheduler.pool if it is defined. + Option(conf.get("mapred.fairscheduler.pool")).foreach { pool => + SharkEnv.sc.setLocalProperty("spark.scheduler.pool", pool) + } + val databaseName = work.databaseName val tableName = work.tableName val hiveTable = Hive.get(conf).getTable(databaseName, tableName) From 1af019c2f79c6e3cc1834fa37f4280356dd04642 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 12 Nov 2013 14:40:06 -0800 Subject: [PATCH 245/331] Clarify comments a bit, and even more clean up in SparkLoadTask. --- src/main/scala/shark/Utils.scala | 3 +- .../shark/execution/OperatorFactory.scala | 4 +- .../scala/shark/execution/SharkDDLTask.scala | 2 +- .../scala/shark/execution/SparkLoadTask.scala | 82 +++++++++++-------- .../memstore2/MemoryMetadataManager.scala | 10 ++- src/main/scala/shark/parse/QueryBlock.scala | 6 +- .../parse/SharkDDLSemanticAnalyzer.scala | 9 +- .../parse/SharkLoadSemanticAnalyzer.scala | 23 +++--- .../shark/parse/SharkSemanticAnalyzer.scala | 12 +-- 9 files changed, 83 insertions(+), 68 deletions(-) diff --git a/src/main/scala/shark/Utils.scala b/src/main/scala/shark/Utils.scala index c4c94cfa..136d7bcd 100644 --- a/src/main/scala/shark/Utils.scala +++ b/src/main/scala/shark/Utils.scala @@ -95,7 +95,8 @@ object Utils { } /** - * Returns a filter that accepts files not in the current snapshot of the `path` directory. + * Returns a filter that accepts files not present in the captured snapshot of the `path` + * directory. */ def createSnapshotFilter(path: Path, conf: Configuration): PathFilter = { val fs = path.getFileSystem(conf) diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 228568bd..1e8c65f9 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -79,8 +79,8 @@ object OperatorFactory extends LogHelper { } /** - * Make sure that the FileSinkOperator uses the disk SerDe for deserializing input rows. - * For example, it should use a LazySimpleSerDe to deserialize LazySimpleStructObjectInspectors. + * Returns the operator tree constructed by createSharkFileOutputPlan(), but makes sure that the + * FileSinkOperator uses the `diskSerDe` (usually a LazySimpleSerDe) for deserializing input rows. */ def createUnifiedViewFileOutputPlan( hiveTerminalOp: HOperator[_<:HiveDesc], diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index b1ccd7c8..a37a8534 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -110,7 +110,7 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] // Creates and directly execute a Hive DDLTask to change the table's SerDe property in // the Hive metastore. // The alternatives are to either attach a HiveDDLTask dependent on this SharkDDLTask or to - // copy the necessary code from Hive, both of which are more troublesome... + // copy the necessary code from Hive, both of which are more troublesome to do... HiveUtils.alterSerdeInHive( dbName, tableName, diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 5d3e1506..f832c6f3 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -62,7 +62,8 @@ class SparkLoadWork( var partSpecs: Seq[JavaMap[String, String]] = Nil def addPartSpec(partSpec: JavaMap[String, String]) { - // Not the most efficient, but this method isn't called very often. + // Not the most efficient, but this method isn't called very often - either a single partition + // spec is passed for partition update, or all partitions are passed for cache load operation. partSpecs = partSpecs ++ Seq(partSpec) } } @@ -74,20 +75,26 @@ object SparkLoadWork { } /** - * Factory/helper method used in LOAD and INSERT INTO/OVERWRITE analysis. This sets all - * necessary fields in SparkLoadWork. + * Factory/helper method used in LOAD and INSERT INTO/OVERWRITE analysis. Sets all necessary + * fields in the SparkLoadWork returned. * * A path filter is created if the command is an INSERT and under these conditions: * - Table is partitioned, and the partition being updated already exists * (i.e., `partSpecOpt.isDefined == true`) - * - Table is not partitioned - Hive is used to check for whether it exists. + * - Table is not partitioned - Hive guarantees that data directories exist for updates on such + * tables. */ def apply( db: Hive, conf: HiveConf, hiveTable: HiveTable, partSpecOpt: Option[JavaMap[String, String]], - commandType: SparkLoadWork.CommandTypes.Type): SparkLoadWork = { + isOverwrite: Boolean): SparkLoadWork = { + val commandType = if (isOverwrite) { + SparkLoadWork.CommandTypes.OVERWRITE + } else { + SparkLoadWork.CommandTypes.INSERT + } val cacheMode = CacheType.fromString(hiveTable.getProperty("shark.cache")) val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( hiveTable.getProperty("shark.cache.storageLevel")) @@ -136,7 +143,8 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val databaseName = work.databaseName val tableName = work.tableName val hiveTable = Hive.get(conf).getTable(databaseName, tableName) - // Used to generate HadoopRDDs. + // Use HadoopTableReader to help with table scans. The `conf` passed is reused across HadoopRDD + // instantiations. val hadoopReader = new HadoopTableReader(Utilities.getTableDesc(hiveTable), conf) if (hiveTable.isPartitioned) { loadPartitionedTable( @@ -155,12 +163,12 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } /** - * Returns a materialized, in-memory RDD comprising TablePartitions backed by columnar stores. + * Returns a materialized, in-memory RDD comprising TablePartitions backed by columnar store. * * @inputRdd A hadoop RDD, or a union of hadoop RDDs if the table is partitioned. - * @serDeProps Properties used to initialize local ColumnarSerDe instantiations. This contains - * the output schema used to create output object inspectors. - * @storageLevel Storage level for the materialized RDD returned. + * @serDeProps Properties used to initialize local ColumnarSerDe instantiations. This contains the + * output schema of the ColumnarSerDe and used to create its output object inspectors. + * @storageLevel Desired persistance level for the materialized RDD returned. * @broadcasedHiveConf Allows for sharing a Hive Configruation broadcast used to create the Hadoop * `inputRdd`. * @inputOI Object inspector used to read rows from `inputRdd`. @@ -191,7 +199,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } } transformedRdd.persist(storageLevel) - // Run a job to materialize the RDD at the cache `storageLevel` specified. + // Run a job to materialize the RDD, persisted at the `storageLevel` given. transformedRdd.context.runJob( transformedRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) (transformedRdd, statsAcc.value) @@ -201,7 +209,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe * Returns Shark MemoryTable that was created or fetched from the metastore, based on the command * type handled by this task. * - * @hiveTable Corresponding HiveTable for which to fetch or create the Shark Memorytable. + * @hiveTable Corresponding HiveTable for which to fetch or create the returned Shark Memorytable. */ def getOrCreateMemoryTable(hiveTable: HiveTable): MemoryTable = { val databaseName = hiveTable.getDbName @@ -215,7 +223,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe work.preferredStorageLevel, work.unifyView) // Before setting the table's SerDe property to ColumnarSerDe, record the SerDe used - // to deserialize rows from disk so that it can be used for successive update operations. + // to deserialize rows from disk so that it can be used for subsequenct update operations. newMemoryTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( databaseName, @@ -243,7 +251,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe * @hadoopReader Used to create a HadoopRDD from the table's data directory. * @pathFilterOpt Defined for INSERT update operations (e.g., INSERT INTO) and passed to * hadoopReader#makeRDDForTable() to determine which new files should be read from the table's - * data directory - see the SparkLoadWork#apply() factory method for an example of how the + * data directory - see the SparkLoadWork#apply() factory method for an example of how a * path filter is created. */ def loadMemoryTable( @@ -275,7 +283,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe // Union the previous and new RDDs, and their respective table stats. val unionedRDD = RDDUtils.unionAndFlatten(tablePartitionRDD, memoryTable.tableRDD) SharkEnv.memoryMetadataManager.getStats(databaseName, tableName ) match { - case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) + case Some(previousStatsMap) => SparkLoadTask.unionStatsMaps(tableStats, previousStatsMap) case None => Unit } unionedRDD @@ -288,7 +296,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe * Returns Shark PartitionedMemorytable that was created or fetched from the metastore, based on * the command type handled by this task. * - * @hiveTable Corresponding HiveTable for the returned Shark PartitionedMemorytable. + * @hiveTable Corresponding HiveTable for the Shark PartitionedMemorytable that will be returned. */ def getOrCreatePartitionedTable( hiveTable: HiveTable, @@ -369,7 +377,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe Map(partition -> partSerDe.getClass), pathFilterOpt) val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( inputRDD, - addPartitionInfoToSerDeProps(partCols, partition.getSchema), + SparkLoadTask.addPartitionInfoToSerDeProps(partCols, partition.getSchema), work.preferredStorageLevel, hadoopReader.broadcastedHiveConf, unionOI) @@ -381,7 +389,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) // Union stats for the previous RDD with the new RDD loaded. SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { - case Some(previousStatsMap) => unionStatsMaps(tableStats, previousStatsMap) + case Some(previousStatsMap) => SparkLoadTask.unionStatsMaps(tableStats, previousStatsMap) case None => Unit } } else { @@ -393,31 +401,42 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } } + override def getType = StageType.MAPRED + + override def getName = "MAPRED-LOAD-SPARK" + + override def localizeMRTmpFilesImpl(ctx: Context) = Unit + +} + +object SparkLoadTask { + /** - * Returns a copy of `baseSerDeProps` with row metadata that contains the names and types for the - * table's partitioning columns. + * Returns a copy of `baseSerDeProps` with the names and types for the table's partitioning + * columns appended to respective row metadata properties. */ def addPartitionInfoToSerDeProps( partCols: Seq[String], baseSerDeProps: Properties): Properties = { val serDeProps = new Properties(baseSerDeProps) // Delimited by "," - var columnNameProperty: String = serDeProps.getProperty(Constants.LIST_COLUMNS) - // NULL if column types are missing. By default, the SerDeParameters initialized by the + var columnNameProperties: String = serDeProps.getProperty(Constants.LIST_COLUMNS) + // `None` if column types are missing. By default, Hive SerDeParameters initialized by the // ColumnarSerDe will treat all columns as having string types. // Delimited by ":" - var columnTypeProperty: String = serDeProps.getProperty(Constants.LIST_COLUMN_TYPES) + var columnTypePropertiesOpt = Option(serDeProps.getProperty(Constants.LIST_COLUMN_TYPES)) for (partColName <- partCols) { - columnNameProperty += "," + partColName + columnNameProperties += "," + partColName } - if (columnTypeProperty != null) { + serDeProps.setProperty(Constants.LIST_COLUMNS, columnNameProperties) + columnTypePropertiesOpt.foreach { columnTypeProperties => + var newColumnTypeProperties = columnTypeProperties for (partColName <- partCols) { - columnTypeProperty += ":" + Constants.STRING_TYPE_NAME + newColumnTypeProperties += ":" + Constants.STRING_TYPE_NAME } + serDeProps.setProperty(Constants.LIST_COLUMN_TYPES, newColumnTypeProperties) } - serDeProps.setProperty(Constants.LIST_COLUMNS, columnNameProperty) - serDeProps.setProperty(Constants.LIST_COLUMN_TYPES, columnTypeProperty) serDeProps } @@ -431,11 +450,4 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } targetStatsMap } - - override def getType = StageType.MAPRED - - override def getName = "MAPRED-LOAD-SPARK" - - override def localizeMRTmpFilesImpl(ctx: Context) = Unit - } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 0e31adad..5e7ae2cb 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -178,8 +178,10 @@ class MemoryMetadataManager extends LogHelper { } /** - * Resets SerDe properties for unified tables to the ones used for deserializing reads. - * That way, tables can be read from disk when the Shark session restarts. + * Resets SerDe properties for unified tables to the ones used for deserializing reads, and clears + * any Shark table properties. + * That way, tables can be read from disk (they'll be indiscernible from Hive tables) when the + * Shark session restarts. */ def resetUnifiedTableSerDes() { val db = Hive.get() @@ -195,13 +197,13 @@ class MemoryMetadataManager extends LogHelper { None /* partitionSpecOpt */, diskSerDe, conf) - // Also remove all Shark related table properties from the Hive table metadata. + // Remove all Shark related table properties from the Hive table metadata. val hiveTable = db.getTable(databaseName, tableName) val tblProps = hiveTable.getParameters tblProps.remove("shark.cache") tblProps.remove("shark.cache.storageLevel") tblProps.remove("shark.cache.unifyView") - // Refresh the Hive db. + // Refresh the Hive `db`/metastore. db.alterTable(tableName, hiveTable) // Reset SerDes if the table is partitioned. sharkTable match { diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 6624f37a..e55a2a74 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -42,7 +42,7 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) // Whether the created to be created or the table specified by CACHED should be backed by disk. private var _unifyView = false - // TableDesc for a table being updated by an INSERT. + // Descriptor for the table being updated by an INSERT. private var _targetTableDesc: TableDesc = _ def cacheModeForCreateTable_= (mode: CacheType) = _cacheModeForCreateTable = mode @@ -57,8 +57,8 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) def unifyView: Boolean = _unifyView - // Hive uses "tableDesc" to refer to the CreateTableDesc. This makes it easier to differentiate - // from `_targetTableDesc`. + // Hive's QB uses `tableDesc` to refer to the CreateTableDesc. A direct `createTableDesc` + // makes it easier to differentiate from `_targetTableDesc`. def createTableDesc: CreateTableDesc = super.getTableDesc def createTableDesc_= (desc: CreateTableDesc) = super.setTableDesc(desc) diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 9b8d858f..441de881 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -69,14 +69,15 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) * If 'true', then create a SparkLoadTask to load the Hive table into memory. * Set it as a dependent of the Hive DDLTask. A SharkDDLTask counterpart isn't created because * the HadoopRDD creation and transformation isn't a direct Shark metastore operation - * (unlike the other cases handled in SharkDDLSemantiAnalyzer). - * - * TODO(harvey): Add "uncache" handling. + * (unlike the other cases handled in SharkDDLSemantiAnalyzer). * * If 'false', then create a SharkDDLTask that will delete the table entry in the Shark * metastore. + * TODO(harvey): Add "uncache" handling. + * * - "shark.cache.unifyView" : * If 'true' and "shark.cache" is true, then the SparkLoadTask created should read this from the * table properties when adding an entry to the Shark metastore. + * * - "shark.cache.storageLevel": * Throw an exception since we can't change the storage level without rescanning the entire RDD. * @@ -93,7 +94,7 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) val oldCacheMode = CacheType.fromString(oldTblProps.get("shark.cache")) val newCacheMode = CacheType.fromString(newTblProps.get("shark.cache")) if (!(oldCacheMode == CacheType.HEAP) && (newCacheMode == CacheType.HEAP)) { - // The table should be cached (and is not already cached) + // The table should be cached (and is not already cached). val partSpecsOpt = if (hiveTable.isPartitioned) { val columnNames = hiveTable.getPartCols.map(_.getName) val partSpecs = db.getPartitions(hiveTable).map { partition => diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index 2c170ba0..21627dc0 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -48,16 +48,15 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con val hiveTable = tableSpec.tableHandle val moveTask = getMoveTask() val partSpecOpt = Option(tableSpec.getPartSpec) - val commandType = if (moveTask.getWork.getLoadTableWork.getReplace) { - SparkLoadWork.CommandTypes.OVERWRITE - } else { - SparkLoadWork.CommandTypes.INSERT - } - val sparkLoadWork = SparkLoadWork(db, conf, hiveTable, partSpecOpt, commandType) + val sparkLoadWork = SparkLoadWork( + db, + conf, + hiveTable, + partSpecOpt, + isOverwrite = moveTask.getWork.getLoadTableWork.getReplace) - // Create a SparkLoadTask that will use a HadoopRDD to read from the source directory. Set it - // to be a dependent task of the LoadTask so that the SparkLoadTask is executed only if the - // Hive task executes successfully. + // Create a SparkLoadTask that will read from the table's data directory. Make it a dependent + // task of the LoadTask so that it's executed only if the LoadTask executes successfully. moveTask.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } } @@ -65,8 +64,8 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con private def getMoveTask(): MoveTask = { assert(rootTasks.size == 1) - // If the execution is local, a CopyTask will be the root task, with a MoveTask child. - // Otherwise, a MoveTask will be the root. + // If the execution is local, then the root task is a CopyTask with a MoveTask child. + // Otherwise, the root is a MoveTask. var rootTask = rootTasks.head val moveTask = if (rootTask.isInstanceOf[CopyTask]) { val firstChildTask = rootTask.getChildTasks.head @@ -76,7 +75,7 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con rootTask } - // In Hive, LoadTableDesc is referred to as LoadTableWork... + // In Hive, LoadTableDesc is referred to as LoadTableWork ... moveTask.asInstanceOf[MoveTask] } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index f4312561..c5fa4fa2 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -399,16 +399,16 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val tableNameSplit = qb.targetTableDesc.getTableName.split('.') val databaseName = tableNameSplit(0) val cachedTableName = tableNameSplit(1) - val commandType = if (qb.getParseInfo.isInsertIntoTable(cachedTableName)) { - SparkLoadWork.CommandTypes.INSERT - } else { - SparkLoadWork.CommandTypes.OVERWRITE - } val hiveTable = db.getTable(databaseName, cachedTableName) // None if the table isn't partitioned, or if the partition specified doesn't exist. val partSpecOpt = Option(qb.getMetaData.getDestPartitionForAlias( qb.getParseInfo.getClauseNamesForDest.head)).map(_.getSpec) - SparkLoadWork(db, conf, hiveTable, partSpecOpt, commandType) + SparkLoadWork( + db, + conf, + hiveTable, + partSpecOpt, + isOverwrite = !qb.getParseInfo.isInsertIntoTable(cachedTableName)) } // Add a SparkLoadTask as a dependent of all MoveTasks, so that when executed, the table's // (or table partition's) data directory will already contain updates that should be From 028c7da8076d143329ae43bd0b64437d4d3fb675 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 12 Nov 2013 21:15:24 -0800 Subject: [PATCH 246/331] Support uncaching at will; table recovery on server failure. --- .../scala/shark/CachedTableRecovery.scala | 91 ------------ src/main/scala/shark/SharkCliDriver.scala | 3 +- src/main/scala/shark/SharkConfVars.scala | 23 +-- src/main/scala/shark/SharkDriver.scala | 25 +--- src/main/scala/shark/SharkEnv.scala | 10 +- src/main/scala/shark/SharkServer.scala | 5 +- .../scala/shark/api/RDDTableFunctions.scala | 15 +- .../execution/MemoryStoreSinkOperator.scala | 7 +- .../scala/shark/execution/SharkDDLTask.scala | 8 +- .../scala/shark/execution/SparkLoadTask.scala | 10 +- .../memstore2/MemoryMetadataManager.scala | 137 ++++++++++++------ .../scala/shark/memstore2/MemoryTable.scala | 7 +- .../memstore2/PartitionedMemoryTable.scala | 7 +- .../shark/memstore2/SharkTblProperties.scala | 79 ++++++++++ src/main/scala/shark/memstore2/Table.scala | 3 +- .../scala/shark/memstore2/TableRecovery.scala | 64 ++++++++ src/main/scala/shark/parse/QueryBlock.scala | 6 + .../parse/SharkDDLSemanticAnalyzer.scala | 37 +++-- .../shark/parse/SharkSemanticAnalyzer.scala | 43 +++--- src/main/scala/shark/util/HiveUtils.scala | 24 +-- .../scala/shark/util/QueryRewriteUtils.scala | 54 +++++++ src/test/scala/shark/SQLSuite.scala | 68 +++++++-- 22 files changed, 478 insertions(+), 248 deletions(-) delete mode 100644 src/main/scala/shark/CachedTableRecovery.scala create mode 100644 src/main/scala/shark/memstore2/SharkTblProperties.scala create mode 100644 src/main/scala/shark/memstore2/TableRecovery.scala create mode 100644 src/main/scala/shark/util/QueryRewriteUtils.scala diff --git a/src/main/scala/shark/CachedTableRecovery.scala b/src/main/scala/shark/CachedTableRecovery.scala deleted file mode 100644 index 4741cf0f..00000000 --- a/src/main/scala/shark/CachedTableRecovery.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import scala.collection.JavaConversions.asScalaBuffer - -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.metadata.Table - - -/** - * Singleton representing access to the Shark Meta data that gets applied to cached tables - * in the Hive Meta Store. - * All cached tables are tagged with a property CTAS_QUERY_STRING whose value - * represents the query that led to the creation of the cached table. - * This is used to reload RDDs upon server restarts. - */ -object CachedTableRecovery extends LogHelper { - - val db = Hive.get(new HiveConf) - - val QUERY_STRING = "CTAS_QUERY_STRING" - - /** - * Load the cached tables into memory. - * @param cmdRunner , the runner that is responsible - * for taking a cached table query and - * a) create the table metadata in Hive Meta Store - * b) load the table as an RDD in memory - * @see SharkServer for an example usage. - */ - def loadAsRdds(cmdRunner: String => Unit) { - getMeta.foreach { t => - try { - db.dropTable(t._1) - cmdRunner(t._2) - } catch { - case e: Exception => logError("Failed to reload cache table " + t._1, e) - } - } - } - - /** - * Updates the Hive metastore, with cached table metadata. - * The cached table metadata is stored in the Hive metastore - * of each cached table, as a key value pair, the key being - * CTAS_QUERY_STRING and the value being the cached table query itself. - * - * @param cachedTableQueries , a collection of pairs of the form - * (cached table name, cached table query). - */ - def updateMeta(cachedTableQueries : Iterable[(String, String)]): Unit = { - cachedTableQueries.foreach { x => - val newTbl = new Table(db.getTable(x._1).getTTable()) - newTbl.setProperty(QUERY_STRING, x._2) - db.alterTable(x._1, newTbl) - } - } - - /** - * Returns all the Cached table metadata present in the Hive Meta store. - * - * @return sequence of pairs, each pair representing the cached table name - * and the cached table query. - */ - def getMeta(): Seq[(String, String)] = { - db.getAllTables().foldLeft(List[(String,String)]())((curr, tableName) => { - val tbl = db.getTable(tableName) - Option(tbl.getProperty(QUERY_STRING)) match { - case Some(q) => curr.::(tableName, q) - case None => curr - } - }) - } -} diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 06e2b3d0..5826d09a 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -44,6 +44,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.io.IOUtils +import shark.memstore2.TableRecovery object SharkCliDriver { @@ -251,7 +252,7 @@ class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper } if (loadRdds) { - CachedTableRecovery.loadAsRdds(processCmd(_)) + TableRecovery.loadUnifiedViews(processCmd(_)) } def this() = this(false) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 519310fa..212eca5d 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -17,6 +17,8 @@ package shark +import java.util.{Map => JavaMap} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf @@ -34,30 +36,15 @@ object SharkConfVars { val COLUMNAR_COMPRESSION = new ConfVar("shark.column.compress", true) + // If true, then cache any table whose name ends in "_cached". + val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) + // Specify the initial capacity for ArrayLists used to represent columns in columnar // cache. The default -1 for non-local mode means that Shark will try to estimate // the number of rows by using: partition_size / (num_columns * avg_field_size). val COLUMN_BUILDER_PARTITION_SIZE = new ConfVar("shark.column.partitionSize.mb", if (System.getenv("MASTER") == null) 1 else -1) - // Default storage level for cached tables. - val STORAGE_LEVEL = new ConfVar("shark.cache.storageLevel", "MEMORY_AND_DISK") - - // Class name of the default cache policy used to manage partition evictions for cached, - // Hive-partitioned tables. - val CACHE_POLICY = new ConfVar( - "shark.cache.policy", "shark.memstore2.CacheAllPolicy") - - // Maximum size - in terms of the number of objects - of the cache specified by the - // "shark.cache.partition.cachePolicy" property above. - val MAX_PARTITION_CACHE_SIZE = new ConfVar("shark.cache.policy.maxSize", "10") - - // Default value for the "shark.cache.unify" table property. - val DEFAULT_UNIFY_FLAG = new ConfVar("shark.cache.unify.default", true) - - // If true, then cache any table whose name ends in "_cached". - val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) - // Prune map splits for cached tables based on predicates in queries. val MAP_PRUNING = new ConfVar("shark.mappruning", true) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 0af2acaf..3c103c1d 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -39,6 +39,7 @@ import shark.execution.{SharkDDLTask, SharkDDLWork, SharkExplainTask, SharkExpla SparkLoadWork, SparkLoadTask, SparkTask, SparkWork} import shark.memstore2.ColumnarSerDe import shark.parse.{QueryContext, SharkSemanticAnalyzerFactory} +import shark.util.QueryRewriteUtils /** @@ -111,8 +112,6 @@ private[shark] object SharkDriver extends LogHelper { def getOp = this.op def getCmd() = this.cmd } - - val CACHE_KEYWORD = "CACHE" } @@ -160,21 +159,6 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe } } - /** - * Rewrites a CACHE command to - * ALTER TABLE SET TBLPROPERTIES ('shark.cache.unifyView' = 'true', 'shark.cache' = 'true'). - */ - def rewriteCacheCmd(cmd: String): String = { - val cmdSplit = cmd.split(' ') - if (cmdSplit.size == 2) { - val tableName = cmdSplit(1) - "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache.unifyView' = 'true', 'shark.cache' = 'true')". - format(tableName) - } else { - throw new SemanticException("CACHE accepts a single table name: 'CACHE
'") - } - } - /** * Overload compile to use Shark's semantic analyzers. */ @@ -198,8 +182,11 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe try { val command = { val varSubbedCmd = new VariableSubstitution().substitute(conf, cmd) - if (varSubbedCmd.toUpperCase.startsWith(SharkDriver.CACHE_KEYWORD)) { - rewriteCacheCmd(varSubbedCmd) + val cmdInUpperCase = varSubbedCmd.toUpperCase + if (cmdInUpperCase.startsWith("CACHE")) { + QueryRewriteUtils.cacheToAlterTable(varSubbedCmd) + } else if (cmdInUpperCase.startsWith("UNCACHE")) { + QueryRewriteUtils.uncacheToAlterTable(varSubbedCmd) } else { varSubbedCmd } diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 4345ce90..890b7f2e 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -17,17 +17,23 @@ package shark +import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME +import org.apache.hadoop.hive.ql.metadata.Hive + import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.StatsReportListener import shark.api.JavaSharkContext -import shark.memstore2.MemoryMetadataManager import shark.execution.serialization.ShuffleSerializer +import shark.memstore2.MemoryMetadataManager import shark.tachyon.TachyonUtilImpl +import org.apache.hadoop.hive.metastore.api.Database + /** A singleton object for the master program. The slaves should not access this. */ object SharkEnv extends LogHelper { @@ -127,7 +133,7 @@ object SharkEnv extends LogHelper { logWarning("Failed to remove table " + tableKey + " from Tachyon."); } } - return memoryMetadataManager.removeTable(databaseName, tableName) + memoryMetadataManager.removeTable(databaseName, tableName) } /** Cleans up and shuts down the Shark environments. */ diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index 67e36044..d112960d 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -53,6 +53,8 @@ import org.apache.thrift.transport.TTransportFactory import org.apache.spark.SparkEnv +import shark.memstore2.TableRecovery + /** * A long-running server compatible with the Hive server. @@ -136,8 +138,7 @@ object SharkServer extends LogHelper { while (!server.isServing()) {} try { val sshandler = new SharkServerHandler - CachedTableRecovery.loadAsRdds(sshandler.execute(_)) - logInfo("Executed load " + CachedTableRecovery.getMeta) + TableRecovery.loadUnifiedViews(sshandler.execute(_)) } catch { case (e: Exception) => logWarning("Unable to load RDDs upon startup", e) } finally { diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index a5fb7866..b94abd87 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -30,7 +30,11 @@ import shark.util.HiveUtils class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { - def saveAsTable(tableName: String, fields: Seq[String], unifyView: Boolean = false): Boolean = { + def saveAsTable( + tableName: String, + fields: Seq[String], + unifyView: Boolean = false, + reloadOnRestart: Boolean = false): Boolean = { require(fields.size == this.manifests.size, "Number of column names != number of fields in the RDD.") @@ -56,14 +60,19 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() - var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) + var isSucessfulCreateTable = HiveUtils.createTableInHive( + tableName, + fields, + manifests, + unifyView, + reloadOnRestart) // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. val databaseName = Hive.get(SharkContext.hiveconf).getCurrentDatabase() if (isSucessfulCreateTable) { // Create an entry in the MemoryMetadataManager. val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, CacheType.HEAP, rdd.getStorageLevel, unifyView) + databaseName, tableName, CacheType.HEAP, rdd.getStorageLevel, unifyView, reloadOnRestart) newTable.tableRDD = rdd try { // Force evaluate to put the data in memory. diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 332cda0d..782cd13f 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -203,7 +203,12 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, cacheMode, storageLevel, unifyView = false)) + databaseName, + tableName, + cacheMode, + storageLevel, + unifyView = false, + reloadOnRestart = false)) memoryTable.tableRDD = outputRDD } diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index a37a8534..f6c4fa63 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -31,6 +31,7 @@ import org.apache.spark.rdd.EmptyRDD import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager, PartitionedMemoryTable} +import shark.memstore2.SharkTblProperties import shark.util.HiveUtils @@ -84,8 +85,8 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val tblProps = createTblDesc.getTblProps val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( tblProps.get("shark.cache.storageLevel")) - val unifyView = tblProps.getOrElse("shark.cache.unifyView", - SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean + val unifyView = tblProps.get(SharkTblProperties.UNIFY_VIEW_FLAG.varname).toBoolean + val reloadOnRestart = tblProps.get(SharkTblProperties.RELOAD_ON_RESTART_FLAG.varname).toBoolean val isHivePartitioned = (createTblDesc.getPartCols.size > 0) val newTable = if (isHivePartitioned) { // Add a new PartitionedMemoryTable entry in the Shark metastore. @@ -96,10 +97,11 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] cacheMode, preferredStorageLevel, unifyView, + reloadOnRestart, tblProps) } else { val memoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( - dbName, tableName, cacheMode, preferredStorageLevel, unifyView) + dbName, tableName, cacheMode, preferredStorageLevel, unifyView, reloadOnRestart) // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. memoryTable.tableRDD = new EmptyRDD(SharkEnv.sc) memoryTable diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index f832c6f3..4960455d 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -54,7 +54,8 @@ class SparkLoadWork( val commandType: SparkLoadWork.CommandTypes.Type, val preferredStorageLevel: StorageLevel, val cacheMode: CacheType.CacheType, - val unifyView: Boolean) + val unifyView: Boolean, + val reloadOnRestart: Boolean) extends java.io.Serializable { var pathFilterOpt: Option[PathFilter] = None @@ -104,7 +105,8 @@ object SparkLoadWork { commandType, preferredStorageLevel, cacheMode, - unifyView = hiveTable.getProperty("shark.cache.unifyView").toBoolean) + unifyView = hiveTable.getProperty("shark.cache.unifyView").toBoolean, + reloadOnRestart = hiveTable.getProperty("shark.cache.reloadOnRestart").toBoolean) partSpecOpt.foreach(sparkLoadWork.addPartSpec(_)) if (commandType == SparkLoadWork.CommandTypes.INSERT) { if (hiveTable.isPartitioned) { @@ -221,7 +223,8 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe tableName, work.cacheMode, work.preferredStorageLevel, - work.unifyView) + work.unifyView, + work.reloadOnRestart) // Before setting the table's SerDe property to ColumnarSerDe, record the SerDe used // to deserialize rows from disk so that it can be used for subsequenct update operations. newMemoryTable.diskSerDe = hiveTable.getDeserializer.getClass.getName @@ -311,6 +314,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe work.cacheMode, work.preferredStorageLevel, work.unifyView, + work.reloadOnRestart, hiveTable.getParameters) newPartitionedTable.diskSerDe = hiveTable.getDeserializer.getClass.getName HiveUtils.alterSerdeInHive( diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 5e7ae2cb..182af4bc 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -77,7 +77,8 @@ class MemoryMetadataManager extends LogHelper { tableName: String, cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, - unifyView: Boolean + unifyView: Boolean, + reloadOnRestart: Boolean ): MemoryTable = { val tableKey = makeTableKey(databaseName, tableName) val newTable = new MemoryTable( @@ -85,7 +86,8 @@ class MemoryMetadataManager extends LogHelper { tableName, cacheMode, preferredStorageLevel, - unifyView) + unifyView, + reloadOnRestart) _keyToTable.put(tableKey, newTable) newTable } @@ -96,6 +98,7 @@ class MemoryMetadataManager extends LogHelper { cacheMode: CacheType.CacheType, preferredStorageLevel: StorageLevel, unifyView: Boolean, + reloadOnRestart: Boolean, tblProps: JavaMap[String, String] ): PartitionedMemoryTable = { val tableKey = makeTableKey(databaseName, tableName) @@ -104,12 +107,13 @@ class MemoryMetadataManager extends LogHelper { tableName, cacheMode, preferredStorageLevel, - unifyView) + unifyView, + reloadOnRestart) // Determine the cache policy to use and read any user-specified cache settings. - val cachePolicyStr = tblProps.getOrElse(SharkConfVars.CACHE_POLICY.varname, - SharkConfVars.CACHE_POLICY.defaultVal) - val maxCacheSize = tblProps.getOrElse(SharkConfVars.MAX_PARTITION_CACHE_SIZE.varname, - SharkConfVars.MAX_PARTITION_CACHE_SIZE.defaultVal).toInt + val cachePolicyStr = tblProps.getOrElse(SharkTblProperties.CACHE_POLICY.varname, + SharkTblProperties.CACHE_POLICY.defaultVal) + val maxCacheSize = tblProps.getOrElse(SharkTblProperties.MAX_PARTITION_CACHE_SIZE.varname, + SharkTblProperties.MAX_PARTITION_CACHE_SIZE.defaultVal).toInt newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize) _keyToTable.put(tableKey, newTable) @@ -174,57 +178,94 @@ class MemoryMetadataManager extends LogHelper { } def shutdown() { - resetUnifiedTableSerDes() + processTablesOnShutdown() + } + + def processTablesOnShutdown() { + val db = Hive.get() + for (sharkTable <- _keyToTable.values) { + if (sharkTable.unifyView) { + resetUnifiedViewProperties( + db, + sharkTable.databaseName, + sharkTable.tableName, + sharkTable.diskSerDe, + sharkTable.reloadOnRestart) + } else { + // Drop everything else + HiveUtils.dropTableInHive(sharkTable.tableName, db.getConf) + } + } } /** - * Resets SerDe properties for unified tables to the ones used for deserializing reads, and clears - * any Shark table properties. - * That way, tables can be read from disk (they'll be indiscernible from Hive tables) when the - * Shark session restarts. + * Drops a unified view from the Shark cache. The table is still backed by disk and its metadata + * can be accessible from the Hive metastore. */ - def resetUnifiedTableSerDes() { - val db = Hive.get() - for (sharkTable <- _keyToTable.values.filter(_.unifyView)) { - val conf = db.getConf - val tableName = sharkTable.tableName - val databaseName = sharkTable.databaseName - val diskSerDe = sharkTable.diskSerDe - logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) - HiveUtils.alterSerdeInHive( + def dropUnifiedView( + db: Hive, + databaseName: String, + tableName: String, + preserveRecoveryProps: Boolean = false) { + getTable(databaseName, tableName).foreach { sharkTable => + // Reset Shark table properties (e.g, reset the SerDe). + resetUnifiedViewProperties( + db, databaseName, tableName, - None /* partitionSpecOpt */, - diskSerDe, - conf) - // Remove all Shark related table properties from the Hive table metadata. - val hiveTable = db.getTable(databaseName, tableName) - val tblProps = hiveTable.getParameters - tblProps.remove("shark.cache") - tblProps.remove("shark.cache.storageLevel") - tblProps.remove("shark.cache.unifyView") - // Refresh the Hive `db`/metastore. - db.alterTable(tableName, hiveTable) - // Reset SerDes if the table is partitioned. - sharkTable match { - case partitionedTable: PartitionedMemoryTable => { - for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { - logInfo("Setting SerDe for table %s(partition %s) back to %s.". - format(tableName, hiveKeyStr, serDeName)) - val partitionSpec = MemoryMetadataManager.parseHivePartitionKeyStr(hiveKeyStr) - HiveUtils.alterSerdeInHive( - databaseName, - tableName, - Some(partitionSpec), - serDeName, - conf) - } + sharkTable.diskSerDe, + preserveRecoveryProps) + // Unpersist the table from memory. + removeTable(databaseName, tableName) + } + } + + /** + * Resets SerDe properties for unified table to the ones used for deserializing reads. + * If `preserveRecoveryProps` is true, then Shark properties needed for table recovery won't be + * removed. + * After this method completes, unified views, upon a Shark server restart, can be loaded into + * the cache automatically or read from disk (indiscernible from Hive tables). + */ + def resetUnifiedViewProperties( + db: Hive, + databaseName: String, + tableName: String, + diskSerDe: String, + preserveRecoveryProps: Boolean) { + val conf = db.getConf + logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) + HiveUtils.alterSerdeInHive( + databaseName, + tableName, + None /* partitionSpecOpt */, + diskSerDe, + conf) + // Remove all Shark related table properties from the Hive table metadata. + val hiveTable = db.getTable(databaseName, tableName) + SharkTblProperties.removeSharkProperties(hiveTable.getParameters, preserveRecoveryProps) + // Refresh the Hive `db`. + db.alterTable(tableName, hiveTable) + // Reset SerDes if the table is partitioned. + getTable(databaseName, tableName) match { + case partitionedTable: PartitionedMemoryTable => { + for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { + logInfo("Setting SerDe for table %s(partition %s) back to %s.". + format(tableName, hiveKeyStr, serDeName)) + val partitionSpec = MemoryMetadataManager.parseHivePartitionKeyStr(hiveKeyStr) + HiveUtils.alterSerdeInHive( + databaseName, + tableName, + Some(partitionSpec), + serDeName, + conf) } - case memoryTable: MemoryTable => Unit } + case _ => Unit } } + private def makeTableKey(databaseName: String, tableName: String): String = { (databaseName + '.' + tableName).toLowerCase } @@ -276,7 +317,7 @@ object MemoryMetadataManager { /** Return a StorageLevel corresponding to its String name. */ def getStorageLevelFromString(s: String): StorageLevel = { if (s == null || s == "") { - getStorageLevelFromString(SharkConfVars.STORAGE_LEVEL.defaultVal) + getStorageLevelFromString(SharkTblProperties.STORAGE_LEVEL.defaultVal) } else { s.toUpperCase match { case "NONE" => StorageLevel.NONE diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index accf7a8e..d4d98fff 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -29,9 +29,10 @@ class MemoryTable( databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel, - unifiedView: Boolean) - extends Table(databaseName, tableName, cacheMode, preferredStorageLevel, unifiedView) { + storageLevel: StorageLevel, + unifiedView: Boolean, + reloadOnRestart: Boolean) + extends Table(databaseName, tableName, cacheMode, storageLevel, unifiedView, reloadOnRestart) { // RDD that contains the contents of this table. private var _tableRDD: RDD[TablePartition] = _ diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index a8bb8f5e..a524e7dd 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -40,9 +40,10 @@ class PartitionedMemoryTable( databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel, - unifiedView: Boolean) - extends Table(databaseName, tableName, cacheMode, preferredStorageLevel, unifiedView) { + storageLevel: StorageLevel, + unifiedView: Boolean, + reloadOnRestart: Boolean) + extends Table(databaseName, tableName, cacheMode, storageLevel, unifiedView, reloadOnRestart) { /** * A simple, mutable wrapper for an RDD. This is needed so that a entry maintained by a diff --git a/src/main/scala/shark/memstore2/SharkTblProperties.scala b/src/main/scala/shark/memstore2/SharkTblProperties.scala new file mode 100644 index 00000000..6d1fc988 --- /dev/null +++ b/src/main/scala/shark/memstore2/SharkTblProperties.scala @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.util.{Map => JavaMap} + + +/** + * Collection of static fields and helpers for table properties (i.e., from A + * CREATE TABLE TBLPROPERTIES( ... ) used by Shark. + */ +object SharkTblProperties { + + case class TableProperty(varname: String, defaultVal: String) + + // Default storage level for cached tables. + val STORAGE_LEVEL = new TableProperty("shark.cache.storageLevel", "MEMORY_AND_DISK") + + // Class name of the default cache policy used to manage partition evictions for cached, + // Hive-partitioned tables. + val CACHE_POLICY = new TableProperty( + "shark.cache.policy", "shark.memstore2.CacheAllPolicy") + + // Maximum size - in terms of the number of objects - of the cache specified by the + // "shark.cache.partition.cachePolicy" property above. + val MAX_PARTITION_CACHE_SIZE = new TableProperty("shark.cache.policy.maxSize", "10") + + // Default value for the "shark.cache.unify" table property. + val UNIFY_VIEW_FLAG = new TableProperty("shark.cache.unifyView", "true") + + // Default value for the "shark.cache.reloadOnRestart" table property. + val RELOAD_ON_RESTART_FLAG = new TableProperty("shark.cache.reloadOnRestart", "true") + + // Default value for the "shark.cache" table property + val CACHE_FLAG = new TableProperty("shark.cache", "true") + + def getOrSetDefault(tblProps: JavaMap[String, String], variable: TableProperty): String = { + if (!tblProps.containsKey(variable.varname)) { + tblProps.put(variable.varname, variable.defaultVal) + } + tblProps.get(variable.varname) + } + + /** + * Returns value for the `variable` table property. If a value isn't present in `tblProps`, then + * the default for `variable` will be returned. + */ + def initializeWithDefaults(tblProps: JavaMap[String, String]): JavaMap[String, String] = { + tblProps.put(CACHE_FLAG.varname, CACHE_FLAG.defaultVal) + tblProps.put(UNIFY_VIEW_FLAG.varname, UNIFY_VIEW_FLAG.defaultVal) + tblProps.put(STORAGE_LEVEL.varname, STORAGE_LEVEL.defaultVal) + tblProps.put(RELOAD_ON_RESTART_FLAG.varname, RELOAD_ON_RESTART_FLAG.defaultVal) + tblProps + } + + def removeSharkProperties(tblProps: JavaMap[String, String], preserveRecoveryProps: Boolean) { + tblProps.remove(CACHE_FLAG.varname) + tblProps.remove(UNIFY_VIEW_FLAG.varname) + if (!preserveRecoveryProps) { + tblProps.remove(STORAGE_LEVEL.varname) + tblProps.remove(RELOAD_ON_RESTART_FLAG.varname) + } + } +} diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index c724377d..1d988a99 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -36,7 +36,8 @@ private[shark] abstract class Table( var tableName: String, var cacheMode: CacheType.CacheType, var preferredStorageLevel: StorageLevel, - var unifyView: Boolean) { + var unifyView: Boolean, + var reloadOnRestart: Boolean) { // SerDe used to read from and write to disk. Should only be set for unified views. private var _diskSerDe: String = _ diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala new file mode 100644 index 00000000..f2f92ae1 --- /dev/null +++ b/src/main/scala/shark/memstore2/TableRecovery.scala @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import scala.collection.JavaConversions.asScalaBuffer + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.metadata.Hive + +import shark.{LogHelper, SharkConfVars} +import shark.memstore2.SharkTblProperties +import shark.util.QueryRewriteUtils + +/** + * Singleton used to reload RDDs upon server restarts. + */ +object TableRecovery extends LogHelper { + + val db = Hive.get() + + /** + * Loads any tables with the `SharkTblProperties.RELOAD_ON_RESTART_FLAG` property set to true. + * @param cmdRunner The runner that is responsible for taking a cached table query and + * a) Creating the table metadata in Hive Meta Store + * b) Loading the table as an RDD in memory + * @see SharkServer for an example usage. + */ + def loadUnifiedViews(cmdRunner: String => Unit) { + // Filter for tables that should be reloaded into the cache. + val currentDbName = db.getCurrentDatabase() + for (databaseName <- db.getAllDatabases()) { + for (tableName <- db.getAllTables(databaseName)) { + val tblProps = db.getTable(databaseName, tableName).getParameters + val shouldReload = Option(tblProps.get(SharkTblProperties.RELOAD_ON_RESTART_FLAG.varname)). + exists(_.toBoolean) + if (shouldReload) { + // Alter the table's properties. + val storageLevelStr = SharkTblProperties.getOrSetDefault(tblProps, + SharkTblProperties.STORAGE_LEVEL) + // TODO(harvey): Executing SQL directly is a bit of a cop-out... + val cmd = QueryRewriteUtils.cacheToAlterTable( + "CACHE %s".format(tableName), storageLevelStr) + cmdRunner(cmd) + } + } + } + db.setCurrentDatabase(currentDbName) + } +} diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index e55a2a74..e258fa34 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -37,6 +37,8 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) // The CacheType for the table that will be created from CREATE TABLE/CTAS. private var _cacheModeForCreateTable = CacheType.NONE + private var _reloadOnRestart: Boolean = false + private var _preferredStorageLevel: StorageLevel = StorageLevel.NONE // Whether the created to be created or the table specified by CACHED should be backed by disk. @@ -49,6 +51,10 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) def cacheModeForCreateTable: CacheType = _cacheModeForCreateTable + def reloadOnRestart: Boolean = _reloadOnRestart + + def reloadOnRestart_= (reload: Boolean) = _reloadOnRestart = reload + def preferredStorageLevel_= (storageLevel: StorageLevel) = _preferredStorageLevel = storageLevel def preferredStorageLevel: StorageLevel = _preferredStorageLevel diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 441de881..2157607e 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -28,13 +28,14 @@ import org.apache.hadoop.hive.ql.parse.ASTNode import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer import org.apache.hadoop.hive.ql.parse.HiveParser +import org.apache.hadoop.hive.ql.parse.SemanticException import org.apache.hadoop.hive.ql.plan.{AlterTableDesc, DDLWork} import org.apache.spark.rdd.{UnionRDD, RDD} import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.{SharkDDLWork, SparkLoadWork} -import shark.memstore2.{CacheType, MemoryMetadataManager} +import shark.memstore2.{CacheType, MemoryMetadataManager, SharkTblProperties} class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) with LogHelper { @@ -72,7 +73,6 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) * (unlike the other cases handled in SharkDDLSemantiAnalyzer). * * If 'false', then create a SharkDDLTask that will delete the table entry in the Shark * metastore. - * TODO(harvey): Add "uncache" handling. * * - "shark.cache.unifyView" : * If 'true' and "shark.cache" is true, then the SparkLoadTask created should read this from the @@ -91,9 +91,9 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) val newTblProps = getAlterTblDesc().getProps val oldTblProps = hiveTable.getParameters - val oldCacheMode = CacheType.fromString(oldTblProps.get("shark.cache")) - val newCacheMode = CacheType.fromString(newTblProps.get("shark.cache")) - if (!(oldCacheMode == CacheType.HEAP) && (newCacheMode == CacheType.HEAP)) { + val oldCacheMode = CacheType.fromString(oldTblProps.get(SharkTblProperties.CACHE_FLAG.varname)) + val newCacheMode = CacheType.fromString(newTblProps.get(SharkTblProperties.CACHE_FLAG.varname)) + if (!CacheType.shouldCache(oldCacheMode) && CacheType.shouldCache(newCacheMode)) { // The table should be cached (and is not already cached). val partSpecsOpt = if (hiveTable.isPartitioned) { val columnNames = hiveTable.getPartCols.map(_.getName) @@ -107,21 +107,36 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) } else { None } - val unifyView = newTblProps.getOrElse("shark.cache.unifyView", - SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean - val cacheMode = newCacheMode + newTblProps.put(SharkTblProperties.CACHE_FLAG.varname, newCacheMode.toString) + val unifyView = SharkTblProperties.getOrSetDefault(newTblProps, + SharkTblProperties.UNIFY_VIEW_FLAG).toBoolean + val reloadOnRestart = SharkTblProperties.getOrSetDefault(newTblProps, + SharkTblProperties.RELOAD_ON_RESTART_FLAG).toBoolean val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - newTblProps.get("shark.cache.storageLevel")) + SharkTblProperties.getOrSetDefault(newTblProps, SharkTblProperties.STORAGE_LEVEL)) val sparkLoadWork = new SparkLoadWork( databaseName, tableName, SparkLoadWork.CommandTypes.NEW_ENTRY, preferredStorageLevel, - cacheMode, - unifyView) + newCacheMode, + unifyView, + reloadOnRestart) partSpecsOpt.foreach(partSpecs => sparkLoadWork.partSpecs = partSpecs) rootTasks.head.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } + if (CacheType.shouldCache(oldCacheMode) && !CacheType.shouldCache(newCacheMode)) { + val isUnifiedView = Option(oldTblProps.get(SharkTblProperties.UNIFY_VIEW_FLAG.varname)). + exists(_.toBoolean) + // Uncache the table. + if (isUnifiedView) { + SharkEnv.memoryMetadataManager.dropUnifiedView(db, databaseName, tableName) + } else { + throw new SemanticException( + "Only unified views can be uncached. A memory-only table should be dropped.") + } + } + } def analyzeDropTableOrDropParts(ast: ASTNode) { diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index c5fa4fa2..f7a0ef49 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -40,11 +40,11 @@ import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan._ import org.apache.hadoop.hive.ql.session.SessionState -import shark.{CachedTableRecovery, LogHelper, SharkConfVars, SharkEnv, Utils} +import shark.{LogHelper, SharkConfVars, SharkEnv, Utils} import shark.execution.{HiveDesc, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator} import shark.execution.{SharkDDLWork, SparkLoadWork, SparkWork, TerminalOperator} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} -import shark.memstore2.{MemoryTable, PartitionedMemoryTable} +import shark.memstore2.{MemoryTable, PartitionedMemoryTable, SharkTblProperties, TableRecovery} /** @@ -53,6 +53,8 @@ import shark.memstore2.{MemoryTable, PartitionedMemoryTable} * mapreduce. We want our query plan to stay intact as a single tree. Since * genMapRedTasks is private, we have to overload analyzeInternal() to use our * own genMapRedTasks(). + * + * One day, this will all be deprecated ... */ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with LogHelper { @@ -248,15 +250,15 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // The table being created from CTAS should be cached. Check whether it should be // synchronized with disk (i.e., maintain a unified view) or memory-only. val tblProps = qb.createTableDesc.getTblProps + // TODO(harvey): Set this during analysis val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) + SharkTblProperties.getOrSetDefault(tblProps, SharkTblProperties.STORAGE_LEVEL)) if (qb.unifyView) { // Save the preferred storage level, since it's needed to create a SparkLoadTask in // genMapRedTasks(). qb.preferredStorageLevel = preferredStorageLevel OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head) } else { - tblProps.put(CachedTableRecovery.QUERY_STRING, ctx.getCmd()) OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOps.head, qb.createTableDesc.getTableName, @@ -380,10 +382,9 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with if (qb.unifyView) { // Create a SparkLoadTask used to scan and load disk contents into the cache. val sparkLoadWork = if (qb.isCTAS) { + // For cached tables, Shark-specific table properties should be set in + // analyzeCreateTable(). val tblProps = qb.createTableDesc.getTblProps - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) - val cacheMode = CacheType.fromString(tblProps.get("shark.cache")) // No need to create a filter, since the entire table data directory should be loaded, nor // pass partition specifications, since partitioned tables can't be created from CTAS. @@ -391,9 +392,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.createTableDesc.getDatabaseName, qb.createTableDesc.getTableName, SparkLoadWork.CommandTypes.NEW_ENTRY, - preferredStorageLevel, - cacheMode, - unifyView = true) + qb.preferredStorageLevel, + qb.cacheModeForCreateTable, + qb.unifyView, + qb.reloadOnRestart) } else { // Split from 'databaseName.tableName' val tableNameSplit = qb.targetTableDesc.getTableName.split('.') @@ -491,22 +493,29 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // 1) Table name includes "_cached" or "_tachyon". // 2) The "shark.cache" table property is "true", or the string representation of a supported // cache mode (heap, Tachyon). - var cacheMode = CacheType.fromString(createTableProperties.get("shark.cache")) + var cacheMode = CacheType.fromString(createTableProperties.get( + SharkTblProperties.CACHE_FLAG.varname)) // Continue planning based on the 'cacheMode' read. if (cacheMode == CacheType.HEAP || (checkTableName && tableName.endsWith("_cached"))) { cacheMode = CacheType.HEAP - createTableProperties.put("shark.cache", cacheMode.toString) + createTableProperties.put(SharkTblProperties.CACHE_FLAG.varname, cacheMode.toString) } else if (cacheMode == CacheType.TACHYON || (checkTableName && tableName.endsWith("_tachyon"))) { cacheMode = CacheType.TACHYON - createTableProperties.put("shark.cache", cacheMode.toString) + createTableProperties.put(SharkTblProperties.CACHE_FLAG.varname, cacheMode.toString) } val shouldCache = CacheType.shouldCache(cacheMode) if (shouldCache) { - queryBlock.unifyView = createTableProperties.getOrElse("shark.cache.unifyView", - SharkConfVars.DEFAULT_UNIFY_FLAG.defaultVal).toBoolean - createTableProperties.put("shark.cache.unifyView", queryBlock.unifyView.toString) + // Add Shark table properties to the QueryBlock. + queryBlock.cacheModeForCreateTable = cacheMode + queryBlock.unifyView = SharkTblProperties.getOrSetDefault(createTableProperties, + SharkTblProperties.UNIFY_VIEW_FLAG).toBoolean + queryBlock.reloadOnRestart = SharkTblProperties.getOrSetDefault(createTableProperties, + SharkTblProperties.RELOAD_ON_RESTART_FLAG).toBoolean + queryBlock.preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( + SharkTblProperties.getOrSetDefault(createTableProperties, SharkTblProperties.STORAGE_LEVEL)) + if (!queryBlock.unifyView) { // Directly set the ColumnarSerDe if the table will be stored memory-only. createTableDesc.setSerName(classOf[ColumnarSerDe].getName) @@ -556,7 +565,7 @@ object SharkSemanticAnalyzer extends LogHelper { val destPartition = qb.getMetaData.getDestPartitionForAlias(selectClauseKey) val partitionColumns = destPartition.getTable.getPartCols.map(_.getName) val partitionColumnToValue = destPartition.getSpec - return MemoryMetadataManager.makeHivePartitionKeyStr(partitionColumns, partitionColumnToValue) + MemoryMetadataManager.makeHivePartitionKeyStr(partitionColumns, partitionColumnToValue) } /** diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 53fc6806..a58e6535 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -39,8 +39,9 @@ import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan.AlterTableDesc import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} -import shark.SharkContext +import shark.{SharkConfVars, SharkContext} import shark.api.{DataType, DataTypes} +import shark.memstore2.SharkTblProperties private[shark] object HiveUtils { @@ -92,25 +93,28 @@ private[shark] object HiveUtils { tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]], + unifyView: Boolean = false, + reloadOnRestart: Boolean = false, hiveConf: HiveConf = new HiveConf): Boolean = { val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") } // Setup the create table descriptor with necessary information. - val createTbleDesc = new CreateTableDesc() - createTbleDesc.setTableName(tableName) - createTbleDesc.setCols(new JavaArrayList[FieldSchema](schema)) - createTbleDesc.setTblProps(Map("shark.cache" -> "heap")) - createTbleDesc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") - createTbleDesc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") - createTbleDesc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) - createTbleDesc.setNumBuckets(-1) + val createTableDesc = new CreateTableDesc() + createTableDesc.setTableName(tableName) + createTableDesc.setCols(new JavaArrayList[FieldSchema](schema)) + createTableDesc.setTblProps( + SharkTblProperties.initializeWithDefaults(createTableDesc.getTblProps)) + createTableDesc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") + createTableDesc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") + createTableDesc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) + createTableDesc.setNumBuckets(-1) // Execute the create table against the Hive metastore. val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], new JavaHashSet[WriteEntity], - createTbleDesc) + createTableDesc) val taskExecutionStatus = executeDDLTaskDirectly(ddlWork, hiveConf) taskExecutionStatus == 0 } diff --git a/src/main/scala/shark/util/QueryRewriteUtils.scala b/src/main/scala/shark/util/QueryRewriteUtils.scala new file mode 100644 index 00000000..ff8efa5f --- /dev/null +++ b/src/main/scala/shark/util/QueryRewriteUtils.scala @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.util + +import org.apache.hadoop.hive.ql.parse.SemanticException + +import shark.memstore2.SharkTblProperties + + +object QueryRewriteUtils { + + def cacheToAlterTable( + cmd: String, + storageLevelStr: String = SharkTblProperties.STORAGE_LEVEL.defaultVal): String = { + val cmdSplit = cmd.split(' ') + if (cmdSplit.size == 2) { + val tableName = cmdSplit(1) + """ + ALTER TABLE %s SET TBLPROPERTIES ( + 'shark.cache' = 'true', + 'shark.cache.unifyView' = 'true', + 'shark.cache.reloadOnRestart' = 'true', + 'shark.cache.storageLevel' = '%s') + """.format(tableName, storageLevelStr) + } else { + throw new SemanticException("CACHE accepts a single table name: 'CACHE
'") + } + } + + def uncacheToAlterTable(cmd: String): String = { + val cmdSplit = cmd.split(' ') + if (cmdSplit.size == 2) { + val tableName = cmdSplit(1) + "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache' = 'false')".format(tableName) + } else { + throw new SemanticException("UNCACHE accepts a single table name: 'CACHE
'") + } + } +} diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 0261ef26..97d92717 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConversions._ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.spark.rdd.UnionRDD @@ -39,7 +40,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val WAREHOUSE_PATH = TestUtils.getWarehousePath() val METASTORE_PATH = TestUtils.getMetastorePath() val MASTER = "local" - val DEFAULT_DB_NAME = "default" + val DEFAULT_DB_NAME = DEFAULT_DATABASE_NAME val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" var sc: SharkContext = _ @@ -49,7 +50,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { * Tables accessible by any test in SQLSuite. Their properties should remain constant across * tests. */ - def loadGlobalTables() { + def loadTables() { // test sc.runSql("drop table if exists test") sc.runSql("CREATE TABLE test (key INT, val STRING)") @@ -106,7 +107,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // second db sc.sql("create database if not exists seconddb") - loadGlobalTables() + loadTables() } override def afterAll() { @@ -1137,7 +1138,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// // CACHE and ALTER TABLE commands ////////////////////////////////////////////////////////////////////////////// - test ("ALTER TABLE caches contents of non-partitioned table if 'shark.cache' is set to true") { + test ("ALTER TABLE caches non-partitioned table if 'shark.cache' is set to true") { sc.runSql("drop table if exists unified_load") sc.runSql("create table unified_load as select * from test") sc.runSql("""alter table unified_load set @@ -1146,7 +1147,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists unified_load") } - test ("ALTER TABLE caches all contents of partitioned table if 'shark.cache' is set to true") { + test ("ALTER TABLE caches partitioned table if 'shark.cache' is set to true") { sc.runSql("drop table if exists unified_part_load") sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") @@ -1156,6 +1157,41 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists unified_part_load") } + test ("ALTER TABLE uncaches non-partitioned table if 'shark.cache' is set to false") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("""alter table unified_load set + tblproperties('shark.cache' = 'false', 'shark.cache.unifyView' = 'false')""") + assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_load")) + expectSql("select count(*) from unified_load", "500") + sc.runSql("drop table if exists unified_load") + } + + test ("ALTER TABLE uncaches partitioned table if 'shark.cache' is set to false") { + sc.runSql("drop table if exists unified_part_load") + sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") + sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") + sc.runSql("""alter table unified_part_load set + tblproperties('shark.cache' = 'false')""") + assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_part_load")) + expectSql("select count(*) from unified_part_load", "500") + sc.runSql("drop table if exists unified_part_load") + } + + test ("UNCACHE behaves like ALTER TABLE SET TBLPROPERTIES ...") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("cache unified_load") + // Double check the table properties. + val tableName = "unified_load" + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) + assert(hiveTable.getProperty("shark.cache") == "HEAP") + assert(hiveTable.getProperty("shark.cache.unifyView") == "true") + // Check that the cache and disk contents are synchronized. + expectUnifiedKVTable(tableName) + sc.runSql("drop table if exists unified_load") + } + test ("CACHE behaves like ALTER TABLE SET TBLPROPERTIES ...") { sc.runSql("drop table if exists unified_load") sc.runSql("create table unified_load as select * from test") @@ -1163,7 +1199,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Double check the table properties. val tableName = "unified_load" val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - assert(hiveTable.getProperty("shark.cache") == "true") + assert(hiveTable.getProperty("shark.cache") == "HEAP") assert(hiveTable.getProperty("shark.cache.unifyView") == "true") // Check that the cache and disk contents are synchronized. expectUnifiedKVTable(tableName) @@ -1179,6 +1215,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // should reset SerDes for the SQLSuite-global tables. val globalCachedTableNames = Seq("test_cached", "test_null_cached", "clicks_cached", "users_cached", "test1_cached") + // Number of rows for each cached table. val cachedTableCounts = new Array[String](globalCachedTableNames.size) for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { @@ -1188,28 +1225,35 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(cacheSerDe == columnarSerDeName) cachedTableCounts(i) = cachedCount } - sharkMetastore.resetUnifiedTableSerDes() + sharkMetastore.processTablesOnShutdown() for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) + // Make sure the SerDe has been reset to the one used for deserializing disk reads. val diskSerDe = hiveTable.getDeserializer.getClass.getName assert(diskSerDe != columnarSerDeName, """SerDe for %s wasn't reset across Shark metastore restart. (disk SerDe: %s)""".format(tableName, diskSerDe)) + // Check that the number of rows from the table on disk remains the same. val onDiskCount = sc.sql("select count(*) from %s".format(tableName))(0) val cachedCount = cachedTableCounts(i) assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) - // Make sure that Shark table properties are removed. + + // Make sure that some Shark table properties are removed. val tblProps = hiveTable.getParameters assert(!tblProps.contains("shark.cache"), "'shark.cache' table property should be removed.") - assert(!tblProps.contains("shark.storageLevel"), - "'shark.storageLevel' table property should be removed.") - assert(!tblProps.contains("shark.unifyView"), + assert(!tblProps.contains("shark.cachy.unifyView"), "'shark.unifyView' table property should be removed.") + + // The tables should be recoverable. + assert(tblProps.contains("shark.cache.storageLevel"), + "'shark.cache.storageLevel' needed for table recovery is missing.") + assert(tblProps.contains("shark.cache.reloadOnRestart"), + "'shark.cache.reloadOnRestart' should be true, since this table is marked for recovery.") } // Finally, reload all tables. - loadGlobalTables() + loadTables() } } From e4a11c27c5e8e95a1f169baf44fbc71f48522b54 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 15 Nov 2013 02:34:47 -0800 Subject: [PATCH 247/331] Address initial review from Reynold --- src/main/scala/shark/SharkEnv.scala | 6 ------ src/main/scala/shark/execution/HadoopTableReader.scala | 10 ++++++++++ src/main/scala/shark/execution/RDDUtils.scala | 4 ++-- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 890b7f2e..06c9afb9 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -17,12 +17,8 @@ package shark -import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} -import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME -import org.apache.hadoop.hive.ql.metadata.Hive - import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.StatsReportListener @@ -32,8 +28,6 @@ import shark.execution.serialization.ShuffleSerializer import shark.memstore2.MemoryMetadataManager import shark.tachyon.TachyonUtilImpl -import org.apache.hadoop.hive.metastore.api.Database - /** A singleton object for the master program. The slaves should not access this. */ object SharkEnv extends LogHelper { diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index 6d9088aa..75d7ab80 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -66,6 +66,11 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf /** * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed * RDD that contains deserialized rows. + * + * @param hiveTable Hive metadata for the table being scanned. + * @param filterOpt If defined, then the filter is used to reject files contained in the data + * directory being read. If None, then all files are accepted. + * @param deserializerClass Class of the SerDe used to deserialize Writables read from Hadoop. */ def makeRDDForTable( hiveTable: HiveTable, @@ -114,6 +119,11 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive * tables, a data directory is created for each partition corresponding to keys specified using * 'PARTITION BY'. + * + * @param partitionToDeserializer Mapping from a Hive partition metadata object to the SerDe + * class to use to deserialize input Writables. + * @param filterOpt If defined, then the filter is used to reject files contained in the data + * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 089695c6..7d516d18 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -40,11 +40,11 @@ object RDDUtils { def unionAndFlatten[T: ClassManifest]( rdd: RDD[T], otherRdd: RDD[T]): UnionRDD[T] = { - val otherRdds = otherRdd match { + val otherRdds: Seq[RDD[T]] = otherRdd match { case otherUnionRdd: UnionRDD[_] => otherUnionRdd.rdds case _ => Seq(otherRdd) } - val rdds = rdd match { + val rdds: Seq[RDD[T]] = rdd match { case unionRdd: UnionRDD[_] => unionRdd.rdds case _ => Seq(rdd) } From 8a0838b77ad8da6a27bf5d64e2bb4b01aceb7c1d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 16 Nov 2013 19:07:08 -0800 Subject: [PATCH 248/331] Updates from Reynold's second review pass. --- .../scala/shark/execution/SparkLoadTask.scala | 138 +++++++++++------- src/main/scala/shark/parse/QueryBlock.scala | 30 +--- .../parse/SharkDDLSemanticAnalyzer.scala | 6 +- .../shark/parse/SharkSemanticAnalyzer.scala | 9 +- 4 files changed, 100 insertions(+), 83 deletions(-) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 4960455d..de9c99c2 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -18,6 +18,7 @@ package shark.execution +import java.io.Serializable import java.util.{HashMap => JavaHashMap, Properties, Map => JavaMap} import scala.collection.JavaConversions._ @@ -44,22 +45,50 @@ import org.apache.spark.storage.StorageLevel import shark.{LogHelper, SharkEnv, Utils} import shark.execution.serialization.KryoSerializer import shark.memstore2._ +import shark.parse.QueryBlock import shark.util.HiveUtils +/** + * Container for fields needed during SparkLoadTask execution. + * + * @param databaseName Namespace for the table being handled. + * @param tableName Name of the table being handled. + * @param commandType Type (an enum) of command that will be executed for the target table. See + * SparkLoadWork.CommandTypes for a description of which SQL commands correspond to each type. + * @param preferredStorageLevel Storage level for the RDD loaded into memory. + * @param cacheMode Cache type that the RDD should be stored in (e.g., Spark heap). + * TODO(harvey): Support Tachyon. + */ private[shark] class SparkLoadWork( val databaseName: String, val tableName: String, val commandType: SparkLoadWork.CommandTypes.Type, val preferredStorageLevel: StorageLevel, - val cacheMode: CacheType.CacheType, - val unifyView: Boolean, - val reloadOnRestart: Boolean) - extends java.io.Serializable { - + val cacheMode: CacheType.CacheType) + extends Serializable { + + // Used for CommandTypes.NEW_ENTRY. + // True if the table should be stored both on disk and in memory. + var unifyView: Boolean = _ + + // Used for CommandTypes.NEW_ENTRY + // True if the Shark table entry created should be marked as recoverable. Table properties needed + // reload the table across sessions will be preserved by the server shutdown hook. See + // MemoryMetadataManager#processTablesOnShutdown() for shutdown handling and + // TblProperties#removeSharkProperties() for the properties that are preserved. + var reloadOnRestart: Boolean = _ + + // Defined if the command is an INSERT and under these conditions: + // - Table is partitioned, and the partition being updated already exists + // (i.e., `partSpecOpt.isDefined == true`) + // - Table is not partitioned - Hive guarantees that data directories exist for updates on such + // tables. var pathFilterOpt: Option[PathFilter] = None + // A collection of partition key specifications for partitions to update. Each key is represented + // by a Map of (partitioning column -> value) pairs. var partSpecs: Seq[JavaMap[String, String]] = Nil def addPartSpec(partSpec: JavaMap[String, String]) { @@ -72,18 +101,21 @@ class SparkLoadWork( object SparkLoadWork { object CommandTypes extends Enumeration { type Type = Value + + // Type of commands executed by the SparkLoadTask created from a SparkLoadWork. + // Corresponding SQL commands for each enum: + // - NEW_ENTRY: + // CACHE or ALTER TABLE
SET TBLPROPERTIES('shark.cache' = true ... ) + // - INSERT: + // INSERT INTO TABLE
or LOAD DATA INPATH '...' INTO
+ // - OVERWRITE: + // INSERT OVERWRITE TABLE
or LOAD DATA INPATH '...'' OVERWRITE INTO
val OVERWRITE, INSERT, NEW_ENTRY = Value } /** * Factory/helper method used in LOAD and INSERT INTO/OVERWRITE analysis. Sets all necessary * fields in the SparkLoadWork returned. - * - * A path filter is created if the command is an INSERT and under these conditions: - * - Table is partitioned, and the partition being updated already exists - * (i.e., `partSpecOpt.isDefined == true`) - * - Table is not partitioned - Hive guarantees that data directories exist for updates on such - * tables. */ def apply( db: Hive, @@ -104,9 +136,7 @@ object SparkLoadWork { hiveTable.getTableName, commandType, preferredStorageLevel, - cacheMode, - unifyView = hiveTable.getProperty("shark.cache.unifyView").toBoolean, - reloadOnRestart = hiveTable.getProperty("shark.cache.reloadOnRestart").toBoolean) + cacheMode) partSpecOpt.foreach(sparkLoadWork.addPartSpec(_)) if (commandType == SparkLoadWork.CommandTypes.INSERT) { if (hiveTable.isPartitioned) { @@ -134,9 +164,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe override def execute(driveContext: DriverContext): Int = { logDebug("Executing " + this.getClass.getName) - // Set Spark's job description to be this query. - SharkEnv.sc.setJobDescription("Loading from Hadoop for a(n) " + work.commandType) - // Set the fair scheduler's pool using mapred.fairscheduler.pool if it is defined. Option(conf.get("mapred.fairscheduler.pool")).foreach { pool => SharkEnv.sc.setLocalProperty("spark.scheduler.pool", pool) @@ -144,12 +171,15 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val databaseName = work.databaseName val tableName = work.tableName + // Set Spark's job description to be this query. + SharkEnv.sc.setJobDescription("Updating table %s.%s for a(n) %s" + .format(databaseName, tableName, work.commandType)) val hiveTable = Hive.get(conf).getTable(databaseName, tableName) // Use HadoopTableReader to help with table scans. The `conf` passed is reused across HadoopRDD // instantiations. val hadoopReader = new HadoopTableReader(Utilities.getTableDesc(hiveTable), conf) if (hiveTable.isPartitioned) { - loadPartitionedTable( + loadPartitionedMemoryTable( hiveTable, work.partSpecs, hadoopReader, @@ -237,12 +267,10 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe newMemoryTable } case _ => { - SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { - case Some(table: MemoryTable) => table - case _ => { - throw new Exception("Internal error: cached table being updated doesn't exist.") - } - } + val tableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) + assert(tableOpt.exists(_.isInstanceOf[MemoryTable]), + "Memory table being updated cannot be found in the Shark metastore.") + tableOpt.get.asInstanceOf[MemoryTable] } } } @@ -280,8 +308,8 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe hadoopReader.broadcastedHiveConf, serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) memoryTable.tableRDD = work.commandType match { - case (SparkLoadWork.CommandTypes.OVERWRITE - | SparkLoadWork.CommandTypes.NEW_ENTRY) => tablePartitionRDD + case (SparkLoadWork.CommandTypes.OVERWRITE | SparkLoadWork.CommandTypes.NEW_ENTRY) => + tablePartitionRDD case SparkLoadWork.CommandTypes.INSERT => { // Union the previous and new RDDs, and their respective table stats. val unionedRDD = RDDUtils.unionAndFlatten(tablePartitionRDD, memoryTable.tableRDD) @@ -296,12 +324,14 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } /** - * Returns Shark PartitionedMemorytable that was created or fetched from the metastore, based on - * the command type handled by this task. + * Returns the created (for CommandType.NEW_ENTRY) or fetched (for CommandType.INSERT or + * OVERWRITE) PartitionedMemoryTable corresponding to `partSpecs`. * * @hiveTable Corresponding HiveTable for the Shark PartitionedMemorytable that will be returned. + * @partSpecs A map of (partitioning column -> corresponding value) that uniquely identifies the + * partition being created or updated. */ - def getOrCreatePartitionedTable( + def getOrCreatePartitionedMemoryTable( hiveTable: HiveTable, partSpecs: JavaMap[String, String]): PartitionedMemoryTable = { val databaseName = hiveTable.getDbName @@ -329,8 +359,10 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { case Some(table: PartitionedMemoryTable) => table case _ => { - throw new Exception( - "Internal error: cached, partitioned table for INSERT handling doesn't exist.") + val tableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) + assert(tableOpt.exists(_.isInstanceOf[PartitionedMemoryTable]), + "Partitioned memory table being updated cannot be found in the Shark metastore.") + tableOpt.get.asInstanceOf[PartitionedMemoryTable] } } } @@ -351,7 +383,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe * partition's data directory - see the SparkLoadWork#apply() factory method for an example of * how a path filter is created. */ - def loadPartitionedTable( + def loadPartitionedMemoryTable( hiveTable: HiveTable, partSpecs: Seq[JavaMap[String, String]], hadoopReader: HadoopTableReader, @@ -362,7 +394,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe for (partSpec <- partSpecs) { // Read, materialize, and store a columnar-backed RDD for `partSpec`. - val partitionedTable = getOrCreatePartitionedTable(hiveTable, partSpec) + val partitionedTable = getOrCreatePartitionedMemoryTable(hiveTable, partSpec) val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) val partition = db.getPartition(hiveTable, partSpec, false /* forceCreate */) @@ -389,13 +421,14 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val tableOpt = partitionedTable.getPartition(partitionKey) if (tableOpt.isDefined && (work.commandType == SparkLoadWork.CommandTypes.INSERT)) { val previousRDD = tableOpt.get - partitionedTable.updatePartition( - partitionKey, RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) + partitionedTable.updatePartition(partitionKey, + RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) // Union stats for the previous RDD with the new RDD loaded. - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { - case Some(previousStatsMap) => SparkLoadTask.unionStatsMaps(tableStats, previousStatsMap) - case None => Unit - } + val previousStatsMapOpt = SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) + assert(SharkEnv.memoryMetadataManager.getStats(databaseName, tableName).isDefined, + "Stats for %s.%s should be defined for an INSERT operation, but are missing.". + format(databaseName, tableName)) + SparkLoadTask.unionStatsMaps(tableStats, previousStatsMapOpt.get) } else { partitionedTable.putPartition(partitionKey, tablePartitionRDD) // If a new partition is added, then the table's SerDe should be used by default. @@ -423,22 +456,25 @@ object SparkLoadTask { partCols: Seq[String], baseSerDeProps: Properties): Properties = { val serDeProps = new Properties(baseSerDeProps) - // Delimited by "," + + // Column names specified by the Constants.LIST_COLUMNS key are delimited by ",". + // E.g., for a table created from + // CREATE TABLE page_views(key INT, val BIGINT), PARTITIONED BY (dt STRING, country STRING), + // `columnNameProperties` will be "key,val". We want to append the "dt, country" partition + // column names to it, and reset the Constants.LIST_COLUMNS entry in the SerDe properties. var columnNameProperties: String = serDeProps.getProperty(Constants.LIST_COLUMNS) + columnNameProperties += "," + partCols.mkString(",") + serDeProps.setProperty(Constants.LIST_COLUMNS, columnNameProperties) + // `None` if column types are missing. By default, Hive SerDeParameters initialized by the // ColumnarSerDe will treat all columns as having string types. - // Delimited by ":" - var columnTypePropertiesOpt = Option(serDeProps.getProperty(Constants.LIST_COLUMN_TYPES)) - - for (partColName <- partCols) { - columnNameProperties += "," + partColName - } - serDeProps.setProperty(Constants.LIST_COLUMNS, columnNameProperties) + // Column types specified by the Constants.LIST_COLUMN_TYPES key are delimited by ":" + // E.g., for the CREATE TABLE example above, if `columnTypeProperties` is defined, then it + // will be "int:bigint". Partition columns are strings, so "string:string" should be appended. + val columnTypePropertiesOpt = Option(serDeProps.getProperty(Constants.LIST_COLUMN_TYPES)) columnTypePropertiesOpt.foreach { columnTypeProperties => - var newColumnTypeProperties = columnTypeProperties - for (partColName <- partCols) { - newColumnTypeProperties += ":" + Constants.STRING_TYPE_NAME - } + var newColumnTypeProperties = columnTypeProperties + + (":" + Constants.STRING_TYPE_NAME * partCols.size) serDeProps.setProperty(Constants.LIST_COLUMN_TYPES, newColumnTypeProperties) } serDeProps diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index e258fa34..1eea6c3e 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -35,41 +35,21 @@ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) extends HiveQueryBlock(outerID, alias, isSubQuery) { // The CacheType for the table that will be created from CREATE TABLE/CTAS. - private var _cacheModeForCreateTable = CacheType.NONE + var cacheModeForCreateTable = CacheType.NONE - private var _reloadOnRestart: Boolean = false + var reloadOnRestart: Boolean = false - private var _preferredStorageLevel: StorageLevel = StorageLevel.NONE + var preferredStorageLevel: StorageLevel = StorageLevel.NONE // Whether the created to be created or the table specified by CACHED should be backed by disk. - private var _unifyView = false + var unifyView = false // Descriptor for the table being updated by an INSERT. - private var _targetTableDesc: TableDesc = _ - - def cacheModeForCreateTable_= (mode: CacheType) = _cacheModeForCreateTable = mode - - def cacheModeForCreateTable: CacheType = _cacheModeForCreateTable - - def reloadOnRestart: Boolean = _reloadOnRestart - - def reloadOnRestart_= (reload: Boolean) = _reloadOnRestart = reload - - def preferredStorageLevel_= (storageLevel: StorageLevel) = _preferredStorageLevel = storageLevel - - def preferredStorageLevel: StorageLevel = _preferredStorageLevel - - def unifyView_= (shouldUnify: Boolean) = _unifyView = shouldUnify - - def unifyView: Boolean = _unifyView + var targetTableDesc: TableDesc = _ // Hive's QB uses `tableDesc` to refer to the CreateTableDesc. A direct `createTableDesc` // makes it easier to differentiate from `_targetTableDesc`. def createTableDesc: CreateTableDesc = super.getTableDesc def createTableDesc_= (desc: CreateTableDesc) = super.setTableDesc(desc) - - def targetTableDesc: TableDesc = _targetTableDesc - - def targetTableDesc_= (desc: TableDesc) = _targetTableDesc = desc } diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 2157607e..a95c9ef9 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -119,9 +119,9 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) tableName, SparkLoadWork.CommandTypes.NEW_ENTRY, preferredStorageLevel, - newCacheMode, - unifyView, - reloadOnRestart) + newCacheMode) + sparkLoadWork.unifyView = unifyView + sparkLoadWork.reloadOnRestart = reloadOnRestart partSpecsOpt.foreach(partSpecs => sparkLoadWork.partSpecs = partSpecs) rootTasks.head.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index f7a0ef49..5ef32100 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -388,14 +388,15 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // No need to create a filter, since the entire table data directory should be loaded, nor // pass partition specifications, since partitioned tables can't be created from CTAS. - new SparkLoadWork( + val sparkLoadWork = new SparkLoadWork( qb.createTableDesc.getDatabaseName, qb.createTableDesc.getTableName, SparkLoadWork.CommandTypes.NEW_ENTRY, qb.preferredStorageLevel, - qb.cacheModeForCreateTable, - qb.unifyView, - qb.reloadOnRestart) + qb.cacheModeForCreateTable) + sparkLoadWork.unifyView = qb.unifyView + sparkLoadWork.reloadOnRestart = qb.reloadOnRestart + sparkLoadWork } else { // Split from 'databaseName.tableName' val tableNameSplit = qb.targetTableDesc.getTableName.split('.') From 786ea70b53ddb4651624c99b03a379a6a20722e3 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 16 Nov 2013 19:56:21 -0800 Subject: [PATCH 249/331] Place empty stats for new cached tables created from a "regular" CREATE TABLE (i.e., not a CTAS). ...to conform to the internal invariant that stats always exist for any cached tables. --- src/main/scala/shark/execution/SharkDDLTask.scala | 4 +++- src/main/scala/shark/memstore2/MemoryMetadataManager.scala | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index f6c4fa63..acb23cae 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -31,7 +31,7 @@ import org.apache.spark.rdd.EmptyRDD import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager, PartitionedMemoryTable} -import shark.memstore2.SharkTblProperties +import shark.memstore2.{SharkTblProperties, TablePartitionStats} import shark.util.HiveUtils @@ -106,6 +106,8 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] memoryTable.tableRDD = new EmptyRDD(SharkEnv.sc) memoryTable } + // Add an empty stats entry to the Shark metastore. + SharkEnv.memoryMetadataManager.putStats(dbName, tableName, Map[Int, TablePartitionStats]()) if (unifyView) { val table = hiveMetadataDb.getTable(tableName) newTable.diskSerDe = table.getDeserializer().getClass.getName diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 182af4bc..ab8f1c40 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -39,6 +39,8 @@ class MemoryMetadataManager extends LogHelper { new ConcurrentHashMap[String, Table]() // TODO(harvey): Support stats for Hive-partitioned tables. + // A stats entry must exist for any cached tables created, so the sizes of this and `_keyToTable` + // are always equal. private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] @@ -265,7 +267,7 @@ class MemoryMetadataManager extends LogHelper { } } - + // Returns the key "databaseName.tableName". private def makeTableKey(databaseName: String, tableName: String): String = { (databaseName + '.' + tableName).toLowerCase } From 4a75596d0fd290053f994313ba27e16c180c4cd9 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 16 Nov 2013 23:17:00 -0800 Subject: [PATCH 250/331] Reorder imports in SharkSemanticAnalyzer --- src/main/scala/shark/parse/SharkSemanticAnalyzer.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 5ef32100..569a3242 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -27,12 +27,12 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} import org.apache.hadoop.hive.metastore.Warehouse +import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} import org.apache.hadoop.hive.ql.exec.{DDLTask, FetchTask} import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} -import org.apache.hadoop.hive.ql.exec.MoveTask import org.apache.hadoop.hive.ql.exec.{Operator => HiveOperator} +import org.apache.hadoop.hive.ql.exec.MoveTask import org.apache.hadoop.hive.ql.exec.TaskFactory import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.optimizer.Optimizer From cfb83e057d1781ce6b4e3b01daadcb976c16c8d4 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 17 Nov 2013 04:16:44 -0800 Subject: [PATCH 251/331] Remove lengthy disk vs cache SerDe maintenance. --- .../shark/execution/OperatorFactory.scala | 16 ---- .../scala/shark/execution/SharkDDLTask.scala | 14 --- .../scala/shark/execution/SparkLoadTask.scala | 30 +------ .../shark/execution/TableScanOperator.scala | 22 ++++- .../memstore2/MemoryMetadataManager.scala | 69 +++----------- .../memstore2/PartitionedMemoryTable.scala | 15 ---- src/main/scala/shark/memstore2/Table.scala | 13 +-- .../shark/parse/SharkSemanticAnalyzer.scala | 9 +- src/main/scala/shark/util/HiveUtils.scala | 35 -------- src/test/scala/shark/SQLSuite.scala | 90 +------------------ 10 files changed, 36 insertions(+), 277 deletions(-) diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 1e8c65f9..9ab66f66 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -78,22 +78,6 @@ object OperatorFactory extends LogHelper { _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } - /** - * Returns the operator tree constructed by createSharkFileOutputPlan(), but makes sure that the - * FileSinkOperator uses the `diskSerDe` (usually a LazySimpleSerDe) for deserializing input rows. - */ - def createUnifiedViewFileOutputPlan( - hiveTerminalOp: HOperator[_<:HiveDesc], - diskSerDe: String): TerminalOperator = { - var hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] - val terminalOp = createSharkFileOutputPlan(hiveTerminalOp) - val fileSinkDesc = hiveOp.getConf - val tableDesc = fileSinkDesc.getTableInfo() - val serDe = Class.forName(diskSerDe).asInstanceOf[Class[Deserializer]] - tableDesc.setDeserializerClass(serDe) - terminalOp - } - def createSharkRddOutputPlan(hiveTerminalOp: HOperator[_<:HiveDesc]): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? val hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index acb23cae..b9270eeb 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -108,20 +108,6 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } // Add an empty stats entry to the Shark metastore. SharkEnv.memoryMetadataManager.putStats(dbName, tableName, Map[Int, TablePartitionStats]()) - if (unifyView) { - val table = hiveMetadataDb.getTable(tableName) - newTable.diskSerDe = table.getDeserializer().getClass.getName - // Creates and directly execute a Hive DDLTask to change the table's SerDe property in - // the Hive metastore. - // The alternatives are to either attach a HiveDDLTask dependent on this SharkDDLTask or to - // copy the necessary code from Hive, both of which are more troublesome to do... - HiveUtils.alterSerdeInHive( - dbName, - tableName, - partitionSpecOpt = None, - classOf[ColumnarSerDe].getName, - conf) - } } /** Handles an ALTER TABLE ADD PARTITION. */ diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index de9c99c2..37796fb0 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -255,15 +255,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe work.preferredStorageLevel, work.unifyView, work.reloadOnRestart) - // Before setting the table's SerDe property to ColumnarSerDe, record the SerDe used - // to deserialize rows from disk so that it can be used for subsequenct update operations. - newMemoryTable.diskSerDe = hiveTable.getDeserializer.getClass.getName - HiveUtils.alterSerdeInHive( - databaseName, - tableName, - partitionSpecOpt = None, - classOf[ColumnarSerDe].getName, - conf) newMemoryTable } case _ => { @@ -293,7 +284,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val tableName = hiveTable.getTableName val memoryTable = getOrCreateMemoryTable(hiveTable) val tableSchema = hiveTable.getSchema - val serDe = Class.forName(memoryTable.diskSerDe).newInstance.asInstanceOf[Deserializer] + val serDe = hiveTable.getDeserializer serDe.initialize(conf, tableSchema) // Scan the Hive table's data directory. val inputRDD = hadoopReader.makeRDDForTable( @@ -338,7 +329,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val tableName = hiveTable.getTableName work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { - val newPartitionedTable = SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( + SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( databaseName, tableName, work.cacheMode, @@ -346,14 +337,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe work.unifyView, work.reloadOnRestart, hiveTable.getParameters) - newPartitionedTable.diskSerDe = hiveTable.getDeserializer.getClass.getName - HiveUtils.alterSerdeInHive( - databaseName, - tableName, - Some(partSpecs), - classOf[ColumnarSerDe].getName, - conf) - newPartitionedTable } case _ => { SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) match { @@ -397,12 +380,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val partitionedTable = getOrCreatePartitionedMemoryTable(hiveTable, partSpec) val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) val partition = db.getPartition(hiveTable, partSpec, false /* forceCreate */) - - // Name of the SerDe used to deserialize the partition contents on disk. If the partition - // specified doesn't currently exist, then default to the table's disk SerDe. - val partSerDeName = partitionedTable.getDiskSerDe(partitionKey). - getOrElse(partitionedTable.diskSerDe) - val partSerDe = Class.forName(partSerDeName).newInstance.asInstanceOf[Deserializer] + val partSerDe = partition.getDeserializer() val partSchema = partition.getSchema partSerDe.initialize(conf, partSchema) // Get a UnionStructObjectInspector that unifies the two StructObjectInspectors for the table @@ -431,8 +409,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe SparkLoadTask.unionStatsMaps(tableStats, previousStatsMapOpt.get) } else { partitionedTable.putPartition(partitionKey, tablePartitionRDD) - // If a new partition is added, then the table's SerDe should be used by default. - partitionedTable.setDiskSerDe(partitionKey, partitionedTable.diskSerDe) } SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 7006af5b..37d87c0e 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -37,7 +37,8 @@ import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.optimization.ColumnPruner -import shark.memstore2.{CacheType, MemoryMetadataManager, TablePartition, TablePartitionStats} +import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} +import shark.memstore2.{TablePartition, TablePartitionStats} import shark.util.HiveUtils @@ -46,7 +47,10 @@ import shark.util.HiveUtils */ class TableScanOperator extends TopOperator[TableScanDesc] { + // TODO(harvey): Try to use 'TableDesc' for execution and save 'Table' for analysis/planning. + // Decouple `Table` from TableReader and ColumnPruner. @transient var table: Table = _ + @transient var hiveOp: HiveTableScanOperator = _ // Metadata for Hive-partitions (i.e if the table was created from PARTITION BY). NULL if this @@ -66,21 +70,33 @@ class TableScanOperator extends TopOperator[TableScanDesc] { @BeanProperty var tableDesc: TableDesc = _ + @BeanProperty var isInMemoryTableScan: Boolean = _ + override def initializeOnMaster() { // Create a local copy of the HiveConf that will be assigned job properties and, for disk reads, // broadcasted to slaves. localHConf = new HiveConf(super.hconf) + isInMemoryTableScan = SharkEnv.memoryMetadataManager.containsTable( + table.getDbName, table.getTableName) } override def outputObjectInspector() = { if (parts == null) { - val serializer = tableDesc.getDeserializerClass().newInstance() + val serializer = if (isInMemoryTableScan) { + new ColumnarSerDe + } else { + tableDesc.getDeserializerClass().newInstance() + } serializer.initialize(hconf, tableDesc.getProperties) serializer.getObjectInspector() } else { val partProps = firstConfPartDesc.getProperties() - val partSerDe = firstConfPartDesc.getDeserializerClass().newInstance() + val partSerDe = if (isInMemoryTableScan) { + new ColumnarSerDe + } else { + firstConfPartDesc.getDeserializerClass().newInstance() + } partSerDe.initialize(hconf, partProps) HiveUtils.makeUnionOIForPartitionedTable(partProps, partSerDe) } diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index ab8f1c40..2bac4acc 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -187,22 +187,22 @@ class MemoryMetadataManager extends LogHelper { val db = Hive.get() for (sharkTable <- _keyToTable.values) { if (sharkTable.unifyView) { - resetUnifiedViewProperties( + dropUnifiedView( db, sharkTable.databaseName, sharkTable.tableName, - sharkTable.diskSerDe, sharkTable.reloadOnRestart) } else { - // Drop everything else HiveUtils.dropTableInHive(sharkTable.tableName, db.getConf) } } } /** - * Drops a unified view from the Shark cache. The table is still backed by disk and its metadata - * can be accessible from the Hive metastore. + * Removes Shark table properties and drops a unified view from the Shark cache. However, if + * `preserveRecoveryProps` is true, then Shark properties needed for table recovery won't be + * removed. + * After this method completes, the table can still be scanned from disk. */ def dropUnifiedView( db: Hive, @@ -210,63 +210,16 @@ class MemoryMetadataManager extends LogHelper { tableName: String, preserveRecoveryProps: Boolean = false) { getTable(databaseName, tableName).foreach { sharkTable => - // Reset Shark table properties (e.g, reset the SerDe). - resetUnifiedViewProperties( - db, - databaseName, - tableName, - sharkTable.diskSerDe, - preserveRecoveryProps) - // Unpersist the table from memory. + db.setCurrentDatabase(databaseName) + val hiveTable = db.getTable(databaseName, tableName) + SharkTblProperties.removeSharkProperties(hiveTable.getParameters, preserveRecoveryProps) + // Refresh the Hive `db`. + db.alterTable(tableName, hiveTable) + // Unpersist the table's RDDs from memory. removeTable(databaseName, tableName) } } - /** - * Resets SerDe properties for unified table to the ones used for deserializing reads. - * If `preserveRecoveryProps` is true, then Shark properties needed for table recovery won't be - * removed. - * After this method completes, unified views, upon a Shark server restart, can be loaded into - * the cache automatically or read from disk (indiscernible from Hive tables). - */ - def resetUnifiedViewProperties( - db: Hive, - databaseName: String, - tableName: String, - diskSerDe: String, - preserveRecoveryProps: Boolean) { - val conf = db.getConf - logInfo("Setting SerDe for table %s back to %s.".format(tableName, diskSerDe)) - HiveUtils.alterSerdeInHive( - databaseName, - tableName, - None /* partitionSpecOpt */, - diskSerDe, - conf) - // Remove all Shark related table properties from the Hive table metadata. - val hiveTable = db.getTable(databaseName, tableName) - SharkTblProperties.removeSharkProperties(hiveTable.getParameters, preserveRecoveryProps) - // Refresh the Hive `db`. - db.alterTable(tableName, hiveTable) - // Reset SerDes if the table is partitioned. - getTable(databaseName, tableName) match { - case partitionedTable: PartitionedMemoryTable => { - for ((hiveKeyStr, serDeName) <- partitionedTable.keyToDiskSerDes) { - logInfo("Setting SerDe for table %s(partition %s) back to %s.". - format(tableName, hiveKeyStr, serDeName)) - val partitionSpec = MemoryMetadataManager.parseHivePartitionKeyStr(hiveKeyStr) - HiveUtils.alterSerdeInHive( - databaseName, - tableName, - Some(partitionSpec), - serDeName, - conf) - } - } - case _ => Unit - } - } - // Returns the key "databaseName.tableName". private def makeTableKey(databaseName: String, tableName: String): String = { (databaseName + '.' + tableName).toLowerCase diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index a524e7dd..d8521204 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -60,11 +60,6 @@ class PartitionedMemoryTable( private val _keyToPartitions: ConcurrentMap[String, RDDValue] = new ConcurrentJavaHashMap[String, RDDValue]() - // Map from Hive-partition key to the SerDe name used to deserialize rows read from disk. - // Should only be used for unified views. - private var _keyToDiskSerDes: ConcurrentMap[String, String] = - new ConcurrentJavaHashMap[String, String]() - // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: // `TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")`. @@ -109,7 +104,6 @@ class PartitionedMemoryTable( def removePartition(partitionKey: String): Option[RDD[TablePartition]] = { val rddRemoved = _keyToPartitions.remove(partitionKey) - _keyToDiskSerDes.remove(partitionKey) if (rddRemoved.isDefined) { _cachePolicy.notifyRemove(partitionKey) } @@ -131,13 +125,6 @@ class PartitionedMemoryTable( _cachePolicy = newPolicy } - def setDiskSerDe(partitionKey: String, serDe: String) = { - assert(unifyView, "Setting diskSerDe for %s, but it isn't a unified view.".format(tableName)) - _keyToDiskSerDes.put(partitionKey, serDe) - } - - def getDiskSerDe(partitionKey: String): Option[String] = _keyToDiskSerDes.get(partitionKey) - def cachePolicy: CachePolicy[String, RDDValue] = _cachePolicy /** Returns an immutable view of (partition key -> RDD) mappings to external callers */ @@ -145,6 +132,4 @@ class PartitionedMemoryTable( _keyToPartitions.mapValues(_.rdd).toMap } - /** Returns an immutable view of (partition key -> SerDe name) mappings to external callers */ - def keyToDiskSerDes: collection.immutable.Map[String, String] = _keyToDiskSerDes.toMap } diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 1d988a99..f308549b 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -37,15 +37,4 @@ private[shark] abstract class Table( var cacheMode: CacheType.CacheType, var preferredStorageLevel: StorageLevel, var unifyView: Boolean, - var reloadOnRestart: Boolean) { - - // SerDe used to read from and write to disk. Should only be set for unified views. - private var _diskSerDe: String = _ - - def diskSerDe: String = _diskSerDe - - def diskSerDe_= (newSerDe: String) = { - assert(unifyView, "Setting diskSerDe for %s, but it isn't a unified view.".format(tableName)) - _diskSerDe = newSerDe - } -} + var reloadOnRestart: Boolean) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 569a3242..20bee98b 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -213,14 +213,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.unifyView = true qb.targetTableDesc = tableDesc qb.preferredStorageLevel = preferredStorageLevel - val diskSerDe = table match { - case memoryTable: MemoryTable => memoryTable.diskSerDe - case partitionedTable: PartitionedMemoryTable => { - partitionedTable.getDiskSerDe(hivePartitionKey).getOrElse( - partitionedTable.diskSerDe) - } - } - OperatorFactory.createUnifiedViewFileOutputPlan(hiveSinkOp, diskSerDe) + OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) } else { OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index a58e6535..aa277831 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -135,41 +135,6 @@ private[shark] object HiveUtils { taskExecutionStatus == 0 } - /** - * Directly executes a Hive DDLTask that changes the SerDe property for the table (or table - * partition) entry in the Hive metastore. Returns `true` if successful. - * - * @tableName Name of table being altered. - * @partitionSpecOpt Map of (partition col, partition key) pairs for which the SerDe is being - * altered. `None` if the table isn't Hive-partitioned. - * @serDeName Class name of new SerDe to use. - * @hiveConf Configuration associated with the current SessionState. - */ - def alterSerdeInHive( - databaseName: String, - tableName: String, - partitionSpecOpt: Option[JavaMap[String, String]], - serDeName: String, - hiveConf: HiveConf = new HiveConf): Boolean = { - val partitionSpec = if (partitionSpecOpt.isDefined) { - partitionSpecOpt.get.asInstanceOf[JavaHashMap[String, String]] - } else { - null - } - val alterTableDesc = new AlterTableDesc(AlterTableDesc.AlterTableTypes.ADDSERDE) - alterTableDesc.setOldName(tableName) - alterTableDesc.setSerdeName(serDeName) - alterTableDesc.setPartSpec(partitionSpec) - val db = Hive.get(hiveConf).setCurrentDatabase(databaseName) - - // Execute the SerDe change against the Hive metastore. - val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], - new JavaHashSet[WriteEntity], - alterTableDesc) - val taskExecutionStatus = executeDDLTaskDirectly(ddlWork, hiveConf) - taskExecutionStatus == 0 - } - /** * Creates a DDLTask from the DDLWork given, and directly calls DDLTask#execute(). Returns 0 if * the create table command is executed successfully. diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 97d92717..79af76e1 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -25,13 +25,12 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME -import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.spark.rdd.UnionRDD import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException -import shark.memstore2.{ColumnarSerDe, MemoryMetadataManager, PartitionedMemoryTable} +import shark.memstore2.{MemoryMetadataManager, PartitionedMemoryTable} import shark.util.HiveUtils @@ -910,83 +909,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { e.getMessage.contains("semantic") } - ////////////////////////////////////////////////////////////////////////////// - // HiveUtils methods that modify Hive metastore - // TODO(harvey): Tests for methods that directly create/drop tables. - ////////////////////////////////////////////////////////////////////////////// - - test("HiveUtils: directly alter table's SerDe") { - def getTableSerDeName(tableName: String): String = { - val hiveTable = Hive.get(SharkContext.hiveconf).getTable(tableName) - hiveTable.getDeserializer.getClass.getName - } - - sc.runSql("drop table if exists alter_table_serde") - sc.runSql("create table alter_table_serde (key int, value string)") - val tableName = "alter_table_serde" - val hiveConf = SharkContext.hiveconf - - val oldSerDeName = getTableSerDeName(tableName) - val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName - - // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe. - HiveUtils.alterSerdeInHive( - DEFAULT_DB_NAME, - tableName, - None /* partitionSpecOpt */, - columnarSerDeName, - hiveConf) - assert(getTableSerDeName(tableName) == columnarSerDeName) - - // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive( - DEFAULT_DB_NAME, - tableName, - None /* partitionSpecOpt */, - oldSerDeName, - hiveConf) - assert(getTableSerDeName(tableName) == oldSerDeName) - } - - test("HiveUtils: directly alter table partition's SerDe") { - def getPartitionSerDeName(tableName: String, partSpec: JavaHashMap[String, String]): String = { - // Get Hive metadata objects. - val metastore = Hive.get(SharkContext.hiveconf) - val table = metastore.getTable(tableName) - val partition = metastore.getPartition(table, partSpec, false /* forceCreate */) - partition.getDeserializer.getClass.getName - } - - sc.runSql("drop table if exists alter_part_serde") - sc.runSql("create table alter_part_serde (key int, value string) partitioned by (keypart int)") - sc.runSql("insert into table alter_part_serde partition (keypart = 1) select * from test") - val tableName = "alter_part_serde" - val hiveConf = SharkContext.hiveconf - val partitionSpec = new JavaHashMap[String, String]() - partitionSpec.put("keypart", "1") - val oldSerDeName = getPartitionSerDeName(tableName, partitionSpec) - val columnarSerDeName = classOf[shark.memstore2.ColumnarSerDe].getName - - // Change the SerDe from the default LazySimpleSerDe to ColumnarSerDe - HiveUtils.alterSerdeInHive( - DEFAULT_DB_NAME, - tableName, - Some(partitionSpec), - columnarSerDeName, - hiveConf) - assert(getPartitionSerDeName(tableName, partitionSpec) == columnarSerDeName) - - // Change the SerDe back to LazySimpleSerDe. - HiveUtils.alterSerdeInHive( - DEFAULT_DB_NAME, - tableName, - Some(partitionSpec), - oldSerDeName, - hiveConf) - assert(getPartitionSerDeName(tableName, partitionSpec) == oldSerDeName) - } - - ////////////////////////////////////////////////////////////////////////////// // Creating unified views ////////////////////////////////////////////////////////////////////////////// @@ -1210,9 +1132,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Unified view persistence ////////////////////////////////////////////////////////////////////////////// test ("Unified views persist across Shark metastore shutdowns.") { - val columnarSerDeName = classOf[ColumnarSerDe].getName - // All cached tables are unified by default, so MemoryMetadataManager#resetUnifiedTableSerDes() - // should reset SerDes for the SQLSuite-global tables. val globalCachedTableNames = Seq("test_cached", "test_null_cached", "clicks_cached", "users_cached", "test1_cached") @@ -1221,19 +1140,12 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) val cachedCount = sc.sql("select count(*) from %s".format(tableName))(0) - val cacheSerDe = hiveTable.getDeserializer.getClass.getName - assert(cacheSerDe == columnarSerDeName) cachedTableCounts(i) = cachedCount } sharkMetastore.processTablesOnShutdown() for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - // Make sure the SerDe has been reset to the one used for deserializing disk reads. - val diskSerDe = hiveTable.getDeserializer.getClass.getName - assert(diskSerDe != columnarSerDeName, """SerDe for %s wasn't reset across Shark metastore - restart. (disk SerDe: %s)""".format(tableName, diskSerDe)) - // Check that the number of rows from the table on disk remains the same. val onDiskCount = sc.sql("select count(*) from %s".format(tableName))(0) val cachedCount = cachedTableCounts(i) From 4a33048190c5be721b6d3154eda4797378e06bce Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Sun, 17 Nov 2013 16:22:54 -0800 Subject: [PATCH 252/331] Audit log on tsocket instance --- src/main/scala/shark/SharkServer.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index 67e36044..630050b8 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -21,6 +21,7 @@ import java.io.FileOutputStream import java.io.IOException import java.io.PrintStream import java.io.UnsupportedEncodingException +import java.net.InetSocketAddress import java.util.ArrayList import java.util.{List => JavaList} import java.util.Properties @@ -50,6 +51,7 @@ import org.apache.thrift.server.TThreadPoolServer import org.apache.thrift.transport.TServerSocket import org.apache.thrift.transport.TTransport import org.apache.thrift.transport.TTransportFactory +import org.apache.thrift.transport.TSocket import org.apache.spark.SparkEnv @@ -85,8 +87,15 @@ object SharkServer extends LogHelper { serverTransport = new TServerSocket(cli.port) val hfactory = new ThriftHiveProcessorFactory(null, new HiveConf()) { - override def getProcessor(t: TTransport) = - new ThriftHive.Processor(new GatedSharkServerHandler(latch)) + override def getProcessor(t: TTransport) = { + var remoteClient = "Unknown" + if (t.isInstanceOf[TSocket]) { + remoteClient = t.asInstanceOf[TSocket].getSocket() + .getRemoteSocketAddress().asInstanceOf[InetSocketAddress] + .getAddress().toString() + } + new ThriftHive.Processor(new GatedSharkServerHandler(latch, remoteClient)) + } } val ttServerArgs = new TThreadPoolServer.Args(serverTransport) .processorFactory(hfactory) @@ -161,9 +170,10 @@ object SharkServer extends LogHelper { } -class GatedSharkServerHandler(latch:CountDownLatch) extends SharkServerHandler { +class GatedSharkServerHandler(latch:CountDownLatch, remoteClient:String) extends SharkServerHandler { override def execute(cmd: String): Unit = { latch.await + logInfo("Audit Log: client=" + remoteClient + " cmd=" + cmd) super.execute(cmd) } } From 9f7f53ea6c5dd3e68cb06509c00e4637e8704bd6 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sun, 17 Nov 2013 17:41:05 -0800 Subject: [PATCH 253/331] Remove unused imports --- src/main/scala/shark/SharkConfVars.scala | 2 -- src/main/scala/shark/execution/HadoopTableReader.scala | 1 - src/main/scala/shark/execution/OperatorFactory.scala | 1 - src/main/scala/shark/execution/SharkDDLTask.scala | 4 ++-- src/main/scala/shark/execution/SparkLoadTask.scala | 9 +++------ src/main/scala/shark/execution/TableReader.scala | 2 -- .../scala/shark/memstore2/MemoryMetadataManager.scala | 3 +-- src/main/scala/shark/memstore2/TableRecovery.scala | 5 ++--- .../scala/shark/parse/SharkDDLSemanticAnalyzer.scala | 5 ++--- .../scala/shark/parse/SharkLoadSemanticAnalyzer.scala | 1 - src/main/scala/shark/parse/SharkSemanticAnalyzer.scala | 4 +--- src/main/scala/shark/util/HiveUtils.scala | 4 +--- src/test/scala/shark/SQLSuite.scala | 1 - 13 files changed, 12 insertions(+), 30 deletions(-) diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index 212eca5d..c13adbee 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -17,8 +17,6 @@ package shark -import java.util.{Map => JavaMap} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index 75d7ab80..f78eb51d 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} -import org.apache.hadoop.io.Writable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.SerializableWritable diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index 9ab66f66..a631da23 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -22,7 +22,6 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator, GroupByPreShuffleOperator} import org.apache.hadoop.hive.ql.exec.{Operator => HOperator} import org.apache.hadoop.hive.ql.metadata.HiveException -import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.storage.StorageLevel diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index b9270eeb..2931f967 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -29,8 +29,8 @@ import org.apache.hadoop.hive.ql.plan.api.StageType import org.apache.spark.rdd.EmptyRDD -import shark.{LogHelper, SharkConfVars, SharkEnv} -import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager, PartitionedMemoryTable} +import shark.{LogHelper, SharkEnv} +import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} import shark.memstore2.{SharkTblProperties, TablePartitionStats} import shark.util.HiveUtils diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 37796fb0..9188221f 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -19,20 +19,18 @@ package shark.execution import java.io.Serializable -import java.util.{HashMap => JavaHashMap, Properties, Map => JavaMap} +import java.util.{Properties, Map => JavaMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, Buffer} +import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.fs.PathFilter import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.{Context, DriverContext} import org.apache.hadoop.hive.ql.exec.{Task => HiveTask, Utilities} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table => HiveTable} -import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.plan.api.StageType import org.apache.hadoop.hive.serde.Constants; -import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat} @@ -45,7 +43,6 @@ import org.apache.spark.storage.StorageLevel import shark.{LogHelper, SharkEnv, Utils} import shark.execution.serialization.KryoSerializer import shark.memstore2._ -import shark.parse.QueryBlock import shark.util.HiveUtils diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index e6cd29f9..40cce0d9 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -22,9 +22,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} -import org.apache.spark.SerializableWritable import shark.{LogHelper, SharkEnv} import shark.api.QueryExecutionException diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 2bac4acc..ac9bc106 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -28,12 +28,11 @@ import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.storage.StorageLevel -import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.RDDUtils import shark.util.HiveUtils -class MemoryMetadataManager extends LogHelper { +class MemoryMetadataManager { private val _keyToTable: ConcurrentMap[String, Table] = new ConcurrentHashMap[String, Table]() diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala index f2f92ae1..4a5e14f3 100644 --- a/src/main/scala/shark/memstore2/TableRecovery.scala +++ b/src/main/scala/shark/memstore2/TableRecovery.scala @@ -19,11 +19,9 @@ package shark.memstore2 import scala.collection.JavaConversions.asScalaBuffer -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.metadata.Hive -import shark.{LogHelper, SharkConfVars} -import shark.memstore2.SharkTblProperties +import shark.LogHelper import shark.util.QueryRewriteUtils /** @@ -49,6 +47,7 @@ object TableRecovery extends LogHelper { val shouldReload = Option(tblProps.get(SharkTblProperties.RELOAD_ON_RESTART_FLAG.varname)). exists(_.toBoolean) if (shouldReload) { + logInfo("Reloading %s.%s into memory.".format(databaseName, tableName)) // Alter the table's properties. val storageLevelStr = SharkTblProperties.getOrSetDefault(tblProps, SharkTblProperties.STORAGE_LEVEL) diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index a95c9ef9..bedad748 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -17,10 +17,9 @@ package shark.parse -import java.util.{HashMap => JavaHashMap, Map => JavaMap} +import java.util.{HashMap => JavaHashMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.Buffer import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.TaskFactory @@ -33,7 +32,7 @@ import org.apache.hadoop.hive.ql.plan.{AlterTableDesc, DDLWork} import org.apache.spark.rdd.{UnionRDD, RDD} -import shark.{LogHelper, SharkConfVars, SharkEnv} +import shark.{LogHelper, SharkEnv} import shark.execution.{SharkDDLWork, SparkLoadWork} import shark.memstore2.{CacheType, MemoryMetadataManager, SharkTblProperties} diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index 21627dc0..3e2b6f9a 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -18,7 +18,6 @@ package shark.parse import scala.collection.JavaConversions._ -import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{CopyTask, MoveTask, TaskFactory} import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 20bee98b..38a05f2f 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -18,10 +18,8 @@ package shark.parse import java.util.ArrayList -import java.util.{HashSet => JavaHashSet} import java.util.{List => JavaList} import java.util.{Map => JavaMap} -import java.lang.reflect.Method import scala.collection.JavaConversions._ @@ -31,8 +29,8 @@ import org.apache.hadoop.hive.metastore.Warehouse import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} import org.apache.hadoop.hive.ql.exec.{DDLTask, FetchTask} import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} -import org.apache.hadoop.hive.ql.exec.{Operator => HiveOperator} import org.apache.hadoop.hive.ql.exec.MoveTask +import org.apache.hadoop.hive.ql.exec.{Operator => HiveOperator} import org.apache.hadoop.hive.ql.exec.TaskFactory import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.optimizer.Optimizer diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index aa277831..ddf6f174 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -18,7 +18,7 @@ package shark.util import java.util.{ArrayList => JavaArrayList, Arrays => JavaArrays} -import java.util.{HashMap => JavaHashMap, HashSet => JavaHashSet, Map => JavaMap} +import java.util.{HashSet => JavaHashSet} import java.util.Properties import scala.collection.JavaConversions._ @@ -35,11 +35,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} -import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan.AlterTableDesc import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} -import shark.{SharkConfVars, SharkContext} import shark.api.{DataType, DataTypes} import shark.memstore2.SharkTblProperties diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 79af76e1..1b68b1b8 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.{MemoryMetadataManager, PartitionedMemoryTable} -import shark.util.HiveUtils class SQLSuite extends FunSuite with BeforeAndAfterAll { From e80a2a158a888e9caa0ae053ed7f6f86fe01d3d3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 17 Nov 2013 23:30:09 -0800 Subject: [PATCH 254/331] Return complex types as strings in the SharkContext API. --- src/main/scala/shark/api/Row.scala | 70 ++++++++++++++++++------------ 1 file changed, 42 insertions(+), 28 deletions(-) diff --git a/src/main/scala/shark/api/Row.scala b/src/main/scala/shark/api/Row.scala index 1c1c2c91..e0f34869 100644 --- a/src/main/scala/shark/api/Row.scala +++ b/src/main/scala/shark/api/Row.scala @@ -17,6 +17,9 @@ package shark.api +import org.apache.hadoop.io.Text +import org.apache.hadoop.hive.serde2.ByteStream +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.objectinspector._ @@ -30,14 +33,24 @@ class Row(val rawdata: Any, val colname2indexMap: Map[String, Int], val oi: Stru def apply(field: String): Object = apply(colname2indexMap(field)) def apply(field: Int): Object = { - val ref = oi.getAllStructFieldRefs.get(field) - val data = oi.getStructFieldData(rawdata, ref) - + val ref: StructField = oi.getAllStructFieldRefs.get(field) + val data: Object = oi.getStructFieldData(rawdata, ref) ref.getFieldObjectInspector match { case poi: PrimitiveObjectInspector => poi.getPrimitiveJavaObject(data) - case loi: ListObjectInspector => loi.getList(data) - case moi: MapObjectInspector => moi.getMap(data) - case soi: StructObjectInspector => soi.getStructFieldsDataAsList(data) + case _: ListObjectInspector | _: MapObjectInspector | _: StructObjectInspector => + // For complex types, return the string representation of data. + val stream = new ByteStream.Output() + LazySimpleSerDe.serialize( + stream, // out + data, // obj + ref.getFieldObjectInspector, // objInspector + Array[Byte](1, 2, 3, 4, 5, 6, 7, 8), // separators + 1, // level + new Text(""), // nullSequence + true, // escaped + 92, // escapeChar + Row.needsEscape) // needsEscape + stream.toString } } @@ -98,33 +111,34 @@ class Row(val rawdata: Any, val colname2indexMap: Map[String, Int], val oi: Stru } ///////////////////////////////////////////////////////////////////////////////////////////////// - // Complex data types - // rxin: I am not sure how useful these APIs are since they would expose the Hive internal - // data structure. For example, in the case of an array of strings, getList would actually - // return a List of LazyString. + // Complex data types - only return the string representation of them for now. ///////////////////////////////////////////////////////////////////////////////////////////////// - def getList(field: String): java.util.List[_] = getList(colname2indexMap(field)) + def getList(field: String): String = getList(colname2indexMap(field)) - def getMap(field: String): java.util.Map[_, _] = getMap(colname2indexMap(field)) + def getMap(field: String): String = getMap(colname2indexMap(field)) - def getStruct(field: String): java.util.List[Object] = getStruct(colname2indexMap(field)) + def getStruct(field: String): String = getStruct(colname2indexMap(field)) - def getList(field: Int): java.util.List[_] = { - val ref = oi.getAllStructFieldRefs.get(field) - val data = oi.getStructFieldData(rawdata, ref) - ref.getFieldObjectInspector.asInstanceOf[ListObjectInspector].getList(data) - } + def getList(field: Int): String = apply(field).asInstanceOf[String] - def getMap(field: Int): java.util.Map[_, _] = { - val ref = oi.getAllStructFieldRefs.get(field) - val data = oi.getStructFieldData(rawdata, ref) - ref.getFieldObjectInspector.asInstanceOf[MapObjectInspector].getMap(data) - } + def getMap(field: Int): String = apply(field).asInstanceOf[String] - def getStruct(field: Int): java.util.List[Object] = { - val ref = oi.getAllStructFieldRefs.get(field) - val data = oi.getStructFieldData(rawdata, ref) - ref.getFieldObjectInspector.asInstanceOf[StructObjectInspector].getStructFieldsDataAsList(data) - } + def getStruct(field: Int): String = apply(field).asInstanceOf[String] +} + + +private[shark] object Row { + // For Hive's LazySimpleSerDe + val needsEscape = Array[Boolean]( + false, true, true, true, true, true, true, true, true, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false, false, false, false, true, + false, false, false, false, false, false, false, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false, false, false, false, false, + false, false, false, false, false, false, false, false, false) } From eef68e9985f5753bb5ae3bbbfd3f22c10a3ec278 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Mon, 18 Nov 2013 02:23:04 -0800 Subject: [PATCH 255/331] Canceljob capability through JDBC client --- src/main/scala/shark/SharkCliDriver.scala | 20 ++++++++- src/main/scala/shark/SharkEnv.scala | 2 + src/main/scala/shark/SharkServer.scala | 52 +++++++++++++++++++++-- 3 files changed, 68 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 06e2b3d0..bafbb60f 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -43,13 +43,16 @@ import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorF import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.io.IOUtils +import org.apache.thrift.transport.TSocket; + object SharkCliDriver { private var prompt = "shark" private var prompt2 = " " // when ';' is not yet seen. - + private var transport:TSocket = _ + installSignalHandler() /** @@ -60,7 +63,15 @@ object SharkCliDriver { def installSignalHandler() { HiveInterruptUtils.add(new HiveInterruptCallback { override def interrupt() { - SharkEnv.sc.cancelAllJobs() + // Handle remote execution mode + if (SharkEnv.sc != null) { + SharkEnv.sc.cancelAllJobs() + } else { + if (transport != null) { + // Force closing of TCP connection upon session termination + transport.getSocket().close() + } + } } }) } @@ -198,6 +209,11 @@ object SharkCliDriver { "spacesForString", classOf[String]) spacesForStringMethod.setAccessible(true) + val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") + clientTransportTSocketField.setAccessible(true) + + transport = clientTransportTSocketField.get(ss).asInstanceOf[TSocket] + var ret = 0 var prefix = "" diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 549817f6..ec6d1565 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -94,6 +94,8 @@ object SharkEnv extends LogHelper { executorEnvVars.put("TACHYON_MASTER", getEnv("TACHYON_MASTER")) executorEnvVars.put("TACHYON_WAREHOUSE_PATH", getEnv("TACHYON_WAREHOUSE_PATH")) + val activeSessions = new HashSet[String] + var sc: SparkContext = _ val shuffleSerializerName = classOf[ShuffleSerializer].getName diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index 630050b8..45de3b03 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -94,7 +94,18 @@ object SharkServer extends LogHelper { .getRemoteSocketAddress().asInstanceOf[InetSocketAddress] .getAddress().toString() } - new ThriftHive.Processor(new GatedSharkServerHandler(latch, remoteClient)) + logInfo("Audit Log: Connection Initiated with JDBC client - " + remoteClient) + + val sessionID = remoteClient + '/' + System.currentTimeMillis() + // Add and enable watcher thread + val jdbcSocket = t.asInstanceOf[TSocket].getSocket() + jdbcSocket.setKeepAlive(true) + val watcher = new JDBCWatcher(jdbcSocket, sessionID) + SharkEnv.activeSessions.add(sessionID) + watcher.start() + + new ThriftHive.Processor(new GatedSharkServerHandler(latch, remoteClient, + sessionID)) } } val ttServerArgs = new TThreadPoolServer.Args(serverTransport) @@ -154,6 +165,26 @@ object SharkServer extends LogHelper { } } } + + class JDBCWatcher(sock:java.net.Socket, sessionID:String) extends Thread { + + override def run() { + try { + while (sock.isConnected && SharkEnv.activeSessions.contains(sessionID)) { + sock.getOutputStream().write((new Array[Byte](0)).toArray) + logDebug("Session Socket Alive - " + sessionID) + Thread.sleep(2*1000) + } + } catch { + case ioe: IOException => Unit + } + + + logInfo("Session Socket connection lost, cleaning up - " + sessionID) + SharkEnv.sc.cancelJobGroup(sessionID) + } + + } // Used to parse command line arguments for the server. class SharkServerCliOptions extends HiveServerCli { @@ -170,11 +201,24 @@ object SharkServer extends LogHelper { } -class GatedSharkServerHandler(latch:CountDownLatch, remoteClient:String) extends SharkServerHandler { +class GatedSharkServerHandler(latch:CountDownLatch, remoteClient:String, + sessionID:String) extends SharkServerHandler { override def execute(cmd: String): Unit = { latch.await - logInfo("Audit Log: client=" + remoteClient + " cmd=" + cmd) - super.execute(cmd) + + logInfo("Audit Log: SessionID=" + sessionID + " client=" + remoteClient + " cmd=" + cmd) + + // Handle cancel commands + if (cmd.startsWith("kill ")) { + logInfo("killing group - " + cmd) + val sessionIDToCancel = cmd.split("\\s+|\\s*;").apply(1) + SharkEnv.activeSessions.remove(sessionIDToCancel) + } else { + // Session ID is used as spark job group + // Job groups control cleanup/cancelling of unneeded jobs on connection terminations + SharkEnv.sc.setJobGroup(sessionID, "Session ID = " + sessionID) + super.execute(cmd) + } } } From 82ece8a92cc5ffbaf2235cafbd984ea60f0342ee Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Mon, 18 Nov 2013 13:26:05 -0800 Subject: [PATCH 256/331] Style formatting --- src/main/scala/shark/SharkCliDriver.scala | 4 ++-- src/main/scala/shark/SharkServer.scala | 20 ++++++++++---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index bafbb60f..f76aceea 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -43,7 +43,7 @@ import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorF import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.io.IOUtils -import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TSocket @@ -69,7 +69,7 @@ object SharkCliDriver { } else { if (transport != null) { // Force closing of TCP connection upon session termination - transport.getSocket().close() + transport.getSocket().close() } } } diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index 45de3b03..d4d415e2 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -168,20 +168,20 @@ object SharkServer extends LogHelper { class JDBCWatcher(sock:java.net.Socket, sessionID:String) extends Thread { - override def run() { - try { - while (sock.isConnected && SharkEnv.activeSessions.contains(sessionID)) { - sock.getOutputStream().write((new Array[Byte](0)).toArray) - logDebug("Session Socket Alive - " + sessionID) + override def run() { + try { + while (sock.isConnected && SharkEnv.activeSessions.contains(sessionID)) { + sock.getOutputStream().write((new Array[Byte](0)).toArray) + logDebug("Session Socket Alive - " + sessionID) Thread.sleep(2*1000) } - } catch { - case ioe: IOException => Unit - } + } catch { + case ioe: IOException => Unit + } - logInfo("Session Socket connection lost, cleaning up - " + sessionID) - SharkEnv.sc.cancelJobGroup(sessionID) + logInfo("Session Socket connection lost, cleaning up - " + sessionID) + SharkEnv.sc.cancelJobGroup(sessionID) } } From 86ed842aaa741ff966cec67e4b60da8696bebcd1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Nov 2013 18:13:21 -0800 Subject: [PATCH 257/331] A few code style changes + removing the storage level table property. --- .../scala/shark/api/RDDTableFunctions.scala | 2 +- .../execution/MemoryStoreSinkOperator.scala | 16 +-- .../shark/execution/OperatorFactory.scala | 4 - .../scala/shark/execution/SharkDDLTask.scala | 18 +--- .../scala/shark/execution/SparkLoadTask.scala | 99 ++++++++----------- .../memstore2/MemoryMetadataManager.scala | 76 ++++---------- .../scala/shark/memstore2/MemoryTable.scala | 8 +- .../memstore2/PartitionedMemoryTable.scala | 16 +-- .../shark/memstore2/SharkTblProperties.scala | 8 +- src/main/scala/shark/memstore2/Table.scala | 8 -- .../scala/shark/memstore2/TableRecovery.scala | 8 +- src/main/scala/shark/parse/QueryBlock.scala | 6 +- .../parse/SharkDDLSemanticAnalyzer.scala | 13 +-- .../shark/parse/SharkSemanticAnalyzer.scala | 12 --- src/main/scala/shark/util/HiveUtils.scala | 17 ++-- .../scala/shark/util/QueryRewriteUtils.scala | 9 +- src/test/scala/shark/SQLSuite.scala | 2 +- 17 files changed, 96 insertions(+), 226 deletions(-) diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index b94abd87..9eeaa3a2 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -72,7 +72,7 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { if (isSucessfulCreateTable) { // Create an entry in the MemoryMetadataManager. val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, CacheType.HEAP, rdd.getStorageLevel, unifyView, reloadOnRestart) + databaseName, tableName, CacheType.HEAP, unifyView, reloadOnRestart) newTable.tableRDD = rdd try { // Force evaluate to put the data in memory. diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 782cd13f..622320c7 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -46,9 +46,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { // If true, columnar storage will use compression. @BeanProperty var shouldCompress: Boolean = _ - // Storage level to use for the RDD created and materialized by this sink operator. - @BeanProperty var storageLevel: StorageLevel = _ - // For CTAS, this is the name of the table that is created. For INSERTS, this is the name of // the table that is modified. @BeanProperty var tableName: String = _ @@ -147,16 +144,10 @@ class MemoryStoreSinkOperator extends TerminalOperator { outputRDD, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } else { // Put the table in Spark block manager. - op.logInfo("Putting %sRDD for %s.%s in Spark block manager, %s %s %s %s".format( - if (useUnionRDD) "Union" else "", - databaseName, - tableName, - if (storageLevel.deserialized) "deserialized" else "serialized", - if (storageLevel.useMemory) "in memory" else "", - if (storageLevel.useMemory && storageLevel.useDisk) "and" else "", - if (storageLevel.useDisk) "on disk" else "")) + op.logInfo("Putting %sRDD for %s.%s in Spark block manager".format( + if (useUnionRDD) "Union" else "", databaseName, tableName)) - outputRDD.persist(storageLevel) + outputRDD.persist(StorageLevel.MEMORY_AND_DISK) val queryOutputRDD = outputRDD if (useUnionRDD) { @@ -206,7 +197,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { databaseName, tableName, cacheMode, - storageLevel, unifyView = false, reloadOnRestart = false)) memoryTable.tableRDD = outputRDD diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index a631da23..a6c9323c 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -23,8 +23,6 @@ import org.apache.hadoop.hive.ql.exec.{GroupByPostShuffleOperator, GroupByPreShu import org.apache.hadoop.hive.ql.exec.{Operator => HOperator} import org.apache.hadoop.hive.ql.metadata.HiveException -import org.apache.spark.storage.StorageLevel - import shark.LogHelper import shark.memstore2.CacheType._ @@ -48,7 +46,6 @@ object OperatorFactory extends LogHelper { hiveTerminalOp: HOperator[_<:HiveDesc], tableName: String, databaseName: String, - storageLevel: StorageLevel, numColumns: Int, hivePartitionKey: String, cacheMode: CacheType, @@ -60,7 +57,6 @@ object OperatorFactory extends LogHelper { sinkOp.localHiveOp = hiveOp sinkOp.tableName = tableName sinkOp.databaseName = databaseName - sinkOp.storageLevel = storageLevel sinkOp.numColumns = numColumns sinkOp.cacheMode = cacheMode sinkOp.hivePartitionKey = hivePartitionKey diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 2931f967..9ffa1701 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -80,31 +80,23 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] hiveMetadataDb: Hive, createTblDesc: CreateTableDesc, cacheMode: CacheType.CacheType) { - val dbName = hiveMetadataDb.getCurrentDatabase() + val dbName = hiveMetadataDb.getCurrentDatabase val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - tblProps.get("shark.cache.storageLevel")) val unifyView = tblProps.get(SharkTblProperties.UNIFY_VIEW_FLAG.varname).toBoolean val reloadOnRestart = tblProps.get(SharkTblProperties.RELOAD_ON_RESTART_FLAG.varname).toBoolean + val isHivePartitioned = (createTblDesc.getPartCols.size > 0) - val newTable = if (isHivePartitioned) { + if (isHivePartitioned) { // Add a new PartitionedMemoryTable entry in the Shark metastore. // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( - dbName, - tableName, - cacheMode, - preferredStorageLevel, - unifyView, - reloadOnRestart, - tblProps) + dbName, tableName, cacheMode, unifyView, reloadOnRestart, tblProps) } else { val memoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( - dbName, tableName, cacheMode, preferredStorageLevel, unifyView, reloadOnRestart) + dbName, tableName, cacheMode, unifyView, reloadOnRestart) // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. memoryTable.tableRDD = new EmptyRDD(SharkEnv.sc) - memoryTable } // Add an empty stats entry to the Shark metastore. SharkEnv.memoryMetadataManager.putStats(dbName, tableName, Map[Int, TablePartitionStats]()) diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 9188221f..be038c66 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -53,7 +53,6 @@ import shark.util.HiveUtils * @param tableName Name of the table being handled. * @param commandType Type (an enum) of command that will be executed for the target table. See * SparkLoadWork.CommandTypes for a description of which SQL commands correspond to each type. - * @param preferredStorageLevel Storage level for the RDD loaded into memory. * @param cacheMode Cache type that the RDD should be stored in (e.g., Spark heap). * TODO(harvey): Support Tachyon. */ @@ -62,7 +61,6 @@ class SparkLoadWork( val databaseName: String, val tableName: String, val commandType: SparkLoadWork.CommandTypes.Type, - val preferredStorageLevel: StorageLevel, val cacheMode: CacheType.CacheType) extends Serializable { @@ -126,13 +124,10 @@ object SparkLoadWork { SparkLoadWork.CommandTypes.INSERT } val cacheMode = CacheType.fromString(hiveTable.getProperty("shark.cache")) - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - hiveTable.getProperty("shark.cache.storageLevel")) val sparkLoadWork = new SparkLoadWork( hiveTable.getDbName, hiveTable.getTableName, commandType, - preferredStorageLevel, cacheMode) partSpecOpt.foreach(sparkLoadWork.addPartSpec(_)) if (commandType == SparkLoadWork.CommandTypes.INSERT) { @@ -192,20 +187,19 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } /** - * Returns a materialized, in-memory RDD comprising TablePartitions backed by columnar store. + * Creates and materializes the in-memory, columnar RDD for a given input RDD. * - * @inputRdd A hadoop RDD, or a union of hadoop RDDs if the table is partitioned. - * @serDeProps Properties used to initialize local ColumnarSerDe instantiations. This contains the - * output schema of the ColumnarSerDe and used to create its output object inspectors. - * @storageLevel Desired persistance level for the materialized RDD returned. - * @broadcasedHiveConf Allows for sharing a Hive Configruation broadcast used to create the Hadoop - * `inputRdd`. - * @inputOI Object inspector used to read rows from `inputRdd`. + * @param inputRdd A hadoop RDD, or a union of hadoop RDDs if the table is partitioned. + * @param serDeProps Properties used to initialize local ColumnarSerDe instantiations. This + * contains the output schema of the ColumnarSerDe and used to create its + * output object inspectors. + * @param broadcastedHiveConf Allows for sharing a Hive Configuration broadcast used to create + * the Hadoop `inputRdd`. + * @param inputOI Object inspector used to read rows from `inputRdd`. */ - def transformAndMaterializeInput( + private def materialize( inputRdd: RDD[_], serDeProps: Properties, - storageLevel: StorageLevel, broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], inputOI: StructObjectInspector) = { val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) @@ -220,41 +214,34 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } if (builder == null) { // Empty partition. - statsAcc += Tuple2(partIndex, new TablePartitionStats(Array(), 0)) + statsAcc += Tuple2(partIndex, new TablePartitionStats(Array.empty, 0)) Iterator(new TablePartition(0, Array())) } else { statsAcc += Tuple2(partIndex, builder.asInstanceOf[TablePartitionBuilder].stats) - Iterator(builder.asInstanceOf[TablePartitionBuilder].build) + Iterator(builder.asInstanceOf[TablePartitionBuilder].build()) } } - transformedRdd.persist(storageLevel) - // Run a job to materialize the RDD, persisted at the `storageLevel` given. + // Run a job to materialize the RDD. + transformedRdd.persist(StorageLevel.MEMORY_AND_DISK) transformedRdd.context.runJob( transformedRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) (transformedRdd, statsAcc.value) } - /** - * Returns Shark MemoryTable that was created or fetched from the metastore, based on the command - * type handled by this task. - * - * @hiveTable Corresponding HiveTable for which to fetch or create the returned Shark Memorytable. - */ - def getOrCreateMemoryTable(hiveTable: HiveTable): MemoryTable = { + /** Returns a MemoryTable for the given Hive table. */ + private def getOrCreateMemoryTable(hiveTable: HiveTable): MemoryTable = { val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName work.commandType match { case SparkLoadWork.CommandTypes.NEW_ENTRY => { - val newMemoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, - tableName, - work.cacheMode, - work.preferredStorageLevel, - work.unifyView, - work.reloadOnRestart) - newMemoryTable + // This is a new entry, e.g. we are caching a new table or partition. + // Create a new MemoryTable object and return that. + SharkEnv.memoryMetadataManager.createMemoryTable(databaseName, tableName, work.cacheMode, + work.unifyView, work.reloadOnRestart) } case _ => { + // This is an existing entry (e.g. we are doing insert or insert overwrite). + // Get the MemoryTable object from the metadata manager. val tableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) assert(tableOpt.exists(_.isInstanceOf[MemoryTable]), "Memory table being updated cannot be found in the Shark metastore.") @@ -266,14 +253,14 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe /** * Handles loading data from disk into the Shark cache for non-partitioned tables. * - * @hiveTable Hive metadata object representing the target table. - * @hadoopReader Used to create a HadoopRDD from the table's data directory. - * @pathFilterOpt Defined for INSERT update operations (e.g., INSERT INTO) and passed to + * @param hiveTable Hive metadata object representing the target table. + * @param hadoopReader Used to create a HadoopRDD from the table's data directory. + * @param pathFilterOpt Defined for INSERT update operations (e.g., INSERT INTO) and passed to * hadoopReader#makeRDDForTable() to determine which new files should be read from the table's * data directory - see the SparkLoadWork#apply() factory method for an example of how a * path filter is created. */ - def loadMemoryTable( + private def loadMemoryTable( hiveTable: HiveTable, hadoopReader: HadoopTableReader, pathFilterOpt: Option[PathFilter]) { @@ -289,10 +276,9 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe pathFilterOpt, serDe.getClass) // Transform the HadoopRDD to an RDD[TablePartition]. - val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + val (tablePartitionRDD, tableStats) = materialize( inputRDD, tableSchema, - memoryTable.preferredStorageLevel, hadoopReader.broadcastedHiveConf, serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) memoryTable.tableRDD = work.commandType match { @@ -315,11 +301,11 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe * Returns the created (for CommandType.NEW_ENTRY) or fetched (for CommandType.INSERT or * OVERWRITE) PartitionedMemoryTable corresponding to `partSpecs`. * - * @hiveTable Corresponding HiveTable for the Shark PartitionedMemorytable that will be returned. - * @partSpecs A map of (partitioning column -> corresponding value) that uniquely identifies the - * partition being created or updated. + * @param hiveTable The Hive Table. + * @param partSpecs A map of (partitioning column -> corresponding value) that uniquely + * identifies the partition being created or updated. */ - def getOrCreatePartitionedMemoryTable( + private def getOrCreatePartitionedMemoryTable( hiveTable: HiveTable, partSpecs: JavaMap[String, String]): PartitionedMemoryTable = { val databaseName = hiveTable.getDbName @@ -330,7 +316,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe databaseName, tableName, work.cacheMode, - work.preferredStorageLevel, work.unifyView, work.reloadOnRestart, hiveTable.getParameters) @@ -352,18 +337,18 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe /** * Handles loading data from disk into the Shark cache for non-partitioned tables. * - * @hiveTable Hive metadata object representing the target table. - * @partSpecs Sequence of partition key specifications that contains either a single key, + * @param hiveTable Hive metadata object representing the target table. + * @param partSpecs Sequence of partition key specifications that contains either a single key, * or all of the table's partition keys. This is because only one partition specficiation is * allowed for each append or overwrite command, and new cache entries (i.e, for a CACHE * comand) are full table scans. - * @hadoopReader Used to create a HadoopRDD from each partition's data directory. - * @pathFilterOpt Defined for INSERT update operations (e.g., INSERT INTO) and passed to + * @param hadoopReader Used to create a HadoopRDD from each partition's data directory. + * @param pathFilterOpt Defined for INSERT update operations (e.g., INSERT INTO) and passed to * hadoopReader#makeRDDForTable() to determine which new files should be read from the table * partition's data directory - see the SparkLoadWork#apply() factory method for an example of * how a path filter is created. */ - def loadPartitionedMemoryTable( + private def loadPartitionedMemoryTable( hiveTable: HiveTable, partSpecs: Seq[JavaMap[String, String]], hadoopReader: HadoopTableReader, @@ -386,10 +371,9 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe // Create a HadoopRDD for the file scan. val inputRDD = hadoopReader.makeRDDForPartitionedTable( Map(partition -> partSerDe.getClass), pathFilterOpt) - val (tablePartitionRDD, tableStats) = transformAndMaterializeInput( + val (tablePartitionRDD, tableStats) = materialize( inputRDD, SparkLoadTask.addPartitionInfoToSerDeProps(partCols, partition.getSchema), - work.preferredStorageLevel, hadoopReader.broadcastedHiveConf, unionOI) // Determine how to cache the table RDD created. @@ -416,16 +400,16 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe override def getName = "MAPRED-LOAD-SPARK" override def localizeMRTmpFilesImpl(ctx: Context) = Unit - } + object SparkLoadTask { /** * Returns a copy of `baseSerDeProps` with the names and types for the table's partitioning * columns appended to respective row metadata properties. */ - def addPartitionInfoToSerDeProps( + private def addPartitionInfoToSerDeProps( partCols: Seq[String], baseSerDeProps: Properties): Properties = { val serDeProps = new Properties(baseSerDeProps) @@ -446,14 +430,13 @@ object SparkLoadTask { // will be "int:bigint". Partition columns are strings, so "string:string" should be appended. val columnTypePropertiesOpt = Option(serDeProps.getProperty(Constants.LIST_COLUMN_TYPES)) columnTypePropertiesOpt.foreach { columnTypeProperties => - var newColumnTypeProperties = columnTypeProperties + - (":" + Constants.STRING_TYPE_NAME * partCols.size) - serDeProps.setProperty(Constants.LIST_COLUMN_TYPES, newColumnTypeProperties) + serDeProps.setProperty(Constants.LIST_COLUMN_TYPES, + columnTypeProperties + (":" + Constants.STRING_TYPE_NAME * partCols.size)) } serDeProps } - def unionStatsMaps( + private def unionStatsMaps( targetStatsMap: ArrayBuffer[(Int, TablePartitionStats)], otherStatsMap: Iterable[(Int, TablePartitionStats)] ): ArrayBuffer[(Int, TablePartitionStats)] = { diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index ac9bc106..2b751f9b 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -21,12 +21,11 @@ import java.util.concurrent.ConcurrentHashMap import java.util.{HashMap=> JavaHashMap, Map => JavaMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, ConcurrentMap} +import scala.collection.mutable.ConcurrentMap import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.spark.rdd.{RDD, UnionRDD} -import org.apache.spark.storage.StorageLevel import shark.execution.RDDUtils import shark.util.HiveUtils @@ -34,18 +33,16 @@ import shark.util.HiveUtils class MemoryMetadataManager { - private val _keyToTable: ConcurrentMap[String, Table] = + // Set of tables, from databaseName.tableName to Table object. + private val _tables: ConcurrentMap[String, Table] = new ConcurrentHashMap[String, Table]() // TODO(harvey): Support stats for Hive-partitioned tables. - // A stats entry must exist for any cached tables created, so the sizes of this and `_keyToTable` - // are always equal. + // Set of stats, from databaseName.tableName to the stats. This is guaranteed to have the same + // structure / size as the _tables map. private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - // List of callback functions to execute when the Shark metastore shuts down. - private val _onShutdownCallbacks = new ArrayBuffer[() => Unit] - def putStats( databaseName: String, tableName: String, @@ -63,21 +60,20 @@ class MemoryMetadataManager { def isHivePartitioned(databaseName: String, tableName: String): Boolean = { val tableKey = makeTableKey(databaseName, tableName) - _keyToTable.get(tableKey) match { + _tables.get(tableKey) match { case Some(table) => table.isInstanceOf[PartitionedMemoryTable] case None => false } } def containsTable(databaseName: String, tableName: String): Boolean = { - _keyToTable.contains(makeTableKey(databaseName, tableName)) + _tables.contains(makeTableKey(databaseName, tableName)) } def createMemoryTable( databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel, unifyView: Boolean, reloadOnRestart: Boolean ): MemoryTable = { @@ -86,10 +82,9 @@ class MemoryMetadataManager { databaseName, tableName, cacheMode, - preferredStorageLevel, unifyView, reloadOnRestart) - _keyToTable.put(tableKey, newTable) + _tables.put(tableKey, newTable) newTable } @@ -97,7 +92,6 @@ class MemoryMetadataManager { databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - preferredStorageLevel: StorageLevel, unifyView: Boolean, reloadOnRestart: Boolean, tblProps: JavaMap[String, String] @@ -107,7 +101,6 @@ class MemoryMetadataManager { databaseName, tableName, cacheMode, - preferredStorageLevel, unifyView, reloadOnRestart) // Determine the cache policy to use and read any user-specified cache settings. @@ -117,17 +110,17 @@ class MemoryMetadataManager { SharkTblProperties.MAX_PARTITION_CACHE_SIZE.defaultVal).toInt newTable.setPartitionCachePolicy(cachePolicyStr, maxCacheSize) - _keyToTable.put(tableKey, newTable) + _tables.put(tableKey, newTable) newTable } def getTable(databaseName: String, tableName: String): Option[Table] = { - _keyToTable.get(makeTableKey(databaseName, tableName)) + _tables.get(makeTableKey(databaseName, tableName)) } def getMemoryTable(databaseName: String, tableName: String): Option[MemoryTable] = { val tableKey = makeTableKey(databaseName, tableName) - val tableOpt = _keyToTable.get(tableKey) + val tableOpt = _tables.get(tableKey) if (tableOpt.isDefined) { assert(tableOpt.get.isInstanceOf[MemoryTable], "getMemoryTable() called for a partitioned table.") @@ -139,7 +132,7 @@ class MemoryMetadataManager { databaseName: String, tableName: String): Option[PartitionedMemoryTable] = { val tableKey = makeTableKey(databaseName, tableName) - val tableOpt = _keyToTable.get(tableKey) + val tableOpt = _tables.get(tableKey) if (tableOpt.isDefined) { assert(tableOpt.get.isInstanceOf[PartitionedMemoryTable], "getPartitionedTable() called for a non-partitioned table.") @@ -153,11 +146,11 @@ class MemoryMetadataManager { val newTableKey = makeTableKey(databaseName, newName) val statsValueEntry = _keyToStats.remove(oldTableKey).get - val tableValueEntry = _keyToTable.remove(oldTableKey).get + val tableValueEntry = _tables.remove(oldTableKey).get tableValueEntry.tableName = newTableKey _keyToStats.put(newTableKey, statsValueEntry) - _keyToTable.put(newTableKey, tableValueEntry) + _tables.put(newTableKey, tableValueEntry) } } @@ -174,25 +167,18 @@ class MemoryMetadataManager { // Remove MemoryTable's entry from Shark metadata. _keyToStats.remove(tableKey) - val tableValue: Option[Table] = _keyToTable.remove(tableKey) + val tableValue: Option[Table] = _tables.remove(tableKey) tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) } def shutdown() { - processTablesOnShutdown() - } - - def processTablesOnShutdown() { val db = Hive.get() - for (sharkTable <- _keyToTable.values) { - if (sharkTable.unifyView) { - dropUnifiedView( - db, - sharkTable.databaseName, - sharkTable.tableName, - sharkTable.reloadOnRestart) + for (table <- _tables.values) { + if (table.unifyView) { + dropUnifiedView(db, table.databaseName, table.tableName, table.reloadOnRestart) } else { - HiveUtils.dropTableInHive(sharkTable.tableName, db.getConf) + // XXXX: Why are we dropping Hive tables? + HiveUtils.dropTableInHive(table.tableName, db.getConf) } } } @@ -267,26 +253,4 @@ object MemoryMetadataManager { } partitionSpec } - - /** Return a StorageLevel corresponding to its String name. */ - def getStorageLevelFromString(s: String): StorageLevel = { - if (s == null || s == "") { - getStorageLevelFromString(SharkTblProperties.STORAGE_LEVEL.defaultVal) - } else { - s.toUpperCase match { - case "NONE" => StorageLevel.NONE - case "DISK_ONLY" => StorageLevel.DISK_ONLY - case "DISK_ONLY_2" => StorageLevel.DISK_ONLY_2 - case "MEMORY_ONLY" => StorageLevel.MEMORY_ONLY - case "MEMORY_ONLY_2" => StorageLevel.MEMORY_ONLY_2 - case "MEMORY_ONLY_SER" => StorageLevel.MEMORY_ONLY_SER - case "MEMORY_ONLY_SER_2" => StorageLevel.MEMORY_ONLY_SER_2 - case "MEMORY_AND_DISK" => StorageLevel.MEMORY_AND_DISK - case "MEMORY_AND_DISK_2" => StorageLevel.MEMORY_AND_DISK_2 - case "MEMORY_AND_DISK_SER" => StorageLevel.MEMORY_AND_DISK_SER - case "MEMORY_AND_DISK_SER_2" => StorageLevel.MEMORY_AND_DISK_SER_2 - case _ => throw new IllegalArgumentException("Unrecognized storage level: " + s) - } - } - } } diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index d4d98fff..5ba3c7ca 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -18,21 +18,17 @@ package shark.memstore2 import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel - /** * A metadata container for a table in Shark that's backed by an RDD. */ -private[shark] -class MemoryTable( +private[shark] class MemoryTable( databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - storageLevel: StorageLevel, unifiedView: Boolean, reloadOnRestart: Boolean) - extends Table(databaseName, tableName, cacheMode, storageLevel, unifiedView, reloadOnRestart) { + extends Table(databaseName, tableName, cacheMode, unifiedView, reloadOnRestart) { // RDD that contains the contents of this table. private var _tableRDD: RDD[TablePartition] = _ diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index d8521204..f8729225 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -22,11 +22,11 @@ import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ConcurrentMap -import shark.execution.RDDUtils - import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import shark.execution.RDDUtils + /** * A metadata container for partitioned Shark table backed by RDDs. @@ -40,10 +40,9 @@ class PartitionedMemoryTable( databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - storageLevel: StorageLevel, unifiedView: Boolean, reloadOnRestart: Boolean) - extends Table(databaseName, tableName, cacheMode, storageLevel, unifiedView, reloadOnRestart) { + extends Table(databaseName, tableName, cacheMode, unifiedView, reloadOnRestart) { /** * A simple, mutable wrapper for an RDD. This is needed so that a entry maintained by a @@ -63,7 +62,7 @@ class PartitionedMemoryTable( // The eviction policy for this table's cached Hive-partitions. An example of how this // can be set from the CLI: // `TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")`. - // If 'None', then all partitions will be persisted in memory using the `preferredStorageLevel`. + // If 'None', then all partitions will be put in memory. private var _cachePolicy: CachePolicy[String, RDDValue] = _ def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) @@ -114,12 +113,13 @@ class PartitionedMemoryTable( // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. val loadFunc: String => RDDValue = (partitionKey: String) => { val rddValue = _keyToPartitions.get(partitionKey).get - rddValue.rdd.persist(preferredStorageLevel) + rddValue.rdd.persist(StorageLevel.MEMORY_AND_DISK) rddValue } // The evictionFunc will unpersist the RDD. - val evictionFunc: (String, RDDValue) => Unit = - (partitionKey: String, rddValue) => RDDUtils.unpersistRDD(rddValue.rdd) + val evictionFunc: (String, RDDValue) => Unit = (partitionKey, rddValue) => { + RDDUtils.unpersistRDD(rddValue.rdd) + } val newPolicy = CachePolicy.instantiateWithUserSpecs[String, RDDValue]( cachePolicyStr, fallbackMaxSize, loadFunc, evictionFunc) _cachePolicy = newPolicy diff --git a/src/main/scala/shark/memstore2/SharkTblProperties.scala b/src/main/scala/shark/memstore2/SharkTblProperties.scala index 6d1fc988..f044317c 100644 --- a/src/main/scala/shark/memstore2/SharkTblProperties.scala +++ b/src/main/scala/shark/memstore2/SharkTblProperties.scala @@ -28,13 +28,9 @@ object SharkTblProperties { case class TableProperty(varname: String, defaultVal: String) - // Default storage level for cached tables. - val STORAGE_LEVEL = new TableProperty("shark.cache.storageLevel", "MEMORY_AND_DISK") - // Class name of the default cache policy used to manage partition evictions for cached, // Hive-partitioned tables. - val CACHE_POLICY = new TableProperty( - "shark.cache.policy", "shark.memstore2.CacheAllPolicy") + val CACHE_POLICY = new TableProperty("shark.cache.policy", "shark.memstore2.CacheAllPolicy") // Maximum size - in terms of the number of objects - of the cache specified by the // "shark.cache.partition.cachePolicy" property above. @@ -63,7 +59,6 @@ object SharkTblProperties { def initializeWithDefaults(tblProps: JavaMap[String, String]): JavaMap[String, String] = { tblProps.put(CACHE_FLAG.varname, CACHE_FLAG.defaultVal) tblProps.put(UNIFY_VIEW_FLAG.varname, UNIFY_VIEW_FLAG.defaultVal) - tblProps.put(STORAGE_LEVEL.varname, STORAGE_LEVEL.defaultVal) tblProps.put(RELOAD_ON_RESTART_FLAG.varname, RELOAD_ON_RESTART_FLAG.defaultVal) tblProps } @@ -72,7 +67,6 @@ object SharkTblProperties { tblProps.remove(CACHE_FLAG.varname) tblProps.remove(UNIFY_VIEW_FLAG.varname) if (!preserveRecoveryProps) { - tblProps.remove(STORAGE_LEVEL.varname) tblProps.remove(RELOAD_ON_RESTART_FLAG.varname) } } diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index f308549b..88035e4f 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -17,24 +17,16 @@ package shark.memstore2 -import org.apache.spark.storage.StorageLevel - - /** * A container for table metadata managed by Shark and Spark. Subclasses are responsible for * how RDDs are set, stored, and accessed. * * @param tableName Name of this table. * @param cacheMode Type of memory storage used for the table (e.g., the Spark block manager). - * @param preferredStorageLevel The user-specified storage level for the Shark table's RDD(s). - * This can be different from the actual RDD storage levels at any point in time, depending on - * the the Spark block manager's RDD eviction policy and, for partitioned tables, the - * Hive-partition RDD eviction policy. */ private[shark] abstract class Table( var databaseName: String, var tableName: String, var cacheMode: CacheType.CacheType, - var preferredStorageLevel: StorageLevel, var unifyView: Boolean, var reloadOnRestart: Boolean) diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala index 4a5e14f3..deeae40a 100644 --- a/src/main/scala/shark/memstore2/TableRecovery.scala +++ b/src/main/scala/shark/memstore2/TableRecovery.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2012 The Regents of The University California. + * Copyright (C) 2012 The Regents of The University California. * All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -48,12 +48,8 @@ object TableRecovery extends LogHelper { exists(_.toBoolean) if (shouldReload) { logInfo("Reloading %s.%s into memory.".format(databaseName, tableName)) - // Alter the table's properties. - val storageLevelStr = SharkTblProperties.getOrSetDefault(tblProps, - SharkTblProperties.STORAGE_LEVEL) // TODO(harvey): Executing SQL directly is a bit of a cop-out... - val cmd = QueryRewriteUtils.cacheToAlterTable( - "CACHE %s".format(tableName), storageLevelStr) + val cmd = QueryRewriteUtils.cacheToAlterTable("CACHE %s".format(tableName)) cmdRunner(cmd) } } diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 1eea6c3e..2c99f5da 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -21,8 +21,6 @@ import org.apache.hadoop.hive.ql.parse.{QB => HiveQueryBlock} import org.apache.hadoop.hive.ql.plan.CreateTableDesc import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.spark.storage.StorageLevel - import shark.memstore2.CacheType import shark.memstore2.CacheType._ @@ -32,15 +30,13 @@ import shark.memstore2.CacheType._ * and analyzing ASTs (e.g. in SharkSemanticAnalyzer#analyzeCreateTable()). */ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) - extends HiveQueryBlock(outerID, alias, isSubQuery) { + extends HiveQueryBlock(outerID, alias, isSubQuery) { // The CacheType for the table that will be created from CREATE TABLE/CTAS. var cacheModeForCreateTable = CacheType.NONE var reloadOnRestart: Boolean = false - var preferredStorageLevel: StorageLevel = StorageLevel.NONE - // Whether the created to be created or the table specified by CACHED should be backed by disk. var unifyView = false diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index bedad748..46223428 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -77,9 +77,6 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) * If 'true' and "shark.cache" is true, then the SparkLoadTask created should read this from the * table properties when adding an entry to the Shark metastore. * - * - "shark.cache.storageLevel": - * Throw an exception since we can't change the storage level without rescanning the entire RDD. - * * TODO(harvey): Add this, though reevaluate it too...some Spark RDDs might depend on the old * version of the RDD, so simply dropping it might not work. */ @@ -111,14 +108,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) SharkTblProperties.UNIFY_VIEW_FLAG).toBoolean val reloadOnRestart = SharkTblProperties.getOrSetDefault(newTblProps, SharkTblProperties.RELOAD_ON_RESTART_FLAG).toBoolean - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - SharkTblProperties.getOrSetDefault(newTblProps, SharkTblProperties.STORAGE_LEVEL)) - val sparkLoadWork = new SparkLoadWork( - databaseName, - tableName, - SparkLoadWork.CommandTypes.NEW_ENTRY, - preferredStorageLevel, - newCacheMode) + val sparkLoadWork = new SparkLoadWork(databaseName, tableName, + SparkLoadWork.CommandTypes.NEW_ENTRY, newCacheMode) sparkLoadWork.unifyView = unifyView sparkLoadWork.reloadOnRestart = reloadOnRestart partSpecsOpt.foreach(partSpecs => sparkLoadWork.partSpecs = partSpecs) diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 38a05f2f..5946ec3e 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -51,8 +51,6 @@ import shark.memstore2.{MemoryTable, PartitionedMemoryTable, SharkTblProperties, * mapreduce. We want our query plan to stay intact as a single tree. Since * genMapRedTasks is private, we have to overload analyzeInternal() to use our * own genMapRedTasks(). - * - * One day, this will all be deprecated ... */ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with LogHelper { @@ -193,7 +191,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // If useUnionRDD is true, the sink op is for INSERT INTO. val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) val cacheMode = table.cacheMode - val preferredStorageLevel = table.preferredStorageLevel val isPartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( databaseName, cachedTableName) var hivePartitionKey = if (isPartitioned) { @@ -210,14 +207,12 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // by the genMapRedTasks() call below. Set fields in `qb` that will be needed. qb.unifyView = true qb.targetTableDesc = tableDesc - qb.preferredStorageLevel = preferredStorageLevel OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) } else { OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOp, cachedTableName, databaseName, - preferredStorageLevel, _resSchema.size, /* numColumns */ hivePartitionKey, cacheMode, @@ -242,19 +237,15 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // synchronized with disk (i.e., maintain a unified view) or memory-only. val tblProps = qb.createTableDesc.getTblProps // TODO(harvey): Set this during analysis - val preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - SharkTblProperties.getOrSetDefault(tblProps, SharkTblProperties.STORAGE_LEVEL)) if (qb.unifyView) { // Save the preferred storage level, since it's needed to create a SparkLoadTask in // genMapRedTasks(). - qb.preferredStorageLevel = preferredStorageLevel OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head) } else { OperatorFactory.createSharkMemoryStoreOutputPlan( hiveSinkOps.head, qb.createTableDesc.getTableName, qb.createTableDesc.getDatabaseName, - preferredStorageLevel, _resSchema.size, /* numColumns */ new String, /* hivePartitionKey */ qb.cacheModeForCreateTable, @@ -383,7 +374,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.createTableDesc.getDatabaseName, qb.createTableDesc.getTableName, SparkLoadWork.CommandTypes.NEW_ENTRY, - qb.preferredStorageLevel, qb.cacheModeForCreateTable) sparkLoadWork.unifyView = qb.unifyView sparkLoadWork.reloadOnRestart = qb.reloadOnRestart @@ -505,8 +495,6 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with SharkTblProperties.UNIFY_VIEW_FLAG).toBoolean queryBlock.reloadOnRestart = SharkTblProperties.getOrSetDefault(createTableProperties, SharkTblProperties.RELOAD_ON_RESTART_FLAG).toBoolean - queryBlock.preferredStorageLevel = MemoryMetadataManager.getStorageLevelFromString( - SharkTblProperties.getOrSetDefault(createTableProperties, SharkTblProperties.STORAGE_LEVEL)) if (!queryBlock.unifyView) { // Directly set the ColumnarSerDe if the table will be stored memory-only. diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index ddf6f174..c5bf9888 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -18,7 +18,7 @@ package shark.util import java.util.{ArrayList => JavaArrayList, Arrays => JavaArrays} -import java.util.{HashSet => JavaHashSet} +import java.util.{HashSet => JHashSet} import java.util.Properties import scala.collection.JavaConversions._ @@ -35,7 +35,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.exec.DDLTask import org.apache.hadoop.hive.ql.hooks.{ReadEntity, WriteEntity} -import org.apache.hadoop.hive.ql.plan.AlterTableDesc import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, DDLWork, DropTableDesc} import shark.api.{DataType, DataTypes} @@ -78,7 +77,7 @@ private[shark] object HiveUtils { val partColObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( partColNames, partColObjectInspectors) val oiList = JavaArrays.asList( - partSerDe.getObjectInspector().asInstanceOf[StructObjectInspector], + partSerDe.getObjectInspector.asInstanceOf[StructObjectInspector], partColObjectInspector.asInstanceOf[StructObjectInspector]) // New oi is union of table + partition object inspectors ObjectInspectorFactory.getUnionStructObjectInspector(oiList) @@ -110,10 +109,8 @@ private[shark] object HiveUtils { createTableDesc.setNumBuckets(-1) // Execute the create table against the Hive metastore. - val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], - new JavaHashSet[WriteEntity], - createTableDesc) - val taskExecutionStatus = executeDDLTaskDirectly(ddlWork, hiveConf) + val work = new DDLWork(new JHashSet[ReadEntity], new JHashSet[WriteEntity], createTableDesc) + val taskExecutionStatus = executeDDLTaskDirectly(work, hiveConf) taskExecutionStatus == 0 } @@ -126,10 +123,8 @@ private[shark] object HiveUtils { false /* stringPartitionColumns */) // Execute the drop table against the metastore. - val ddlWork = new DDLWork(new JavaHashSet[ReadEntity], - new JavaHashSet[WriteEntity], - dropTblDesc) - val taskExecutionStatus = executeDDLTaskDirectly(ddlWork, hiveConf) + val work = new DDLWork(new JHashSet[ReadEntity], new JHashSet[WriteEntity], dropTblDesc) + val taskExecutionStatus = executeDDLTaskDirectly(work, hiveConf) taskExecutionStatus == 0 } diff --git a/src/main/scala/shark/util/QueryRewriteUtils.scala b/src/main/scala/shark/util/QueryRewriteUtils.scala index ff8efa5f..de2396ba 100644 --- a/src/main/scala/shark/util/QueryRewriteUtils.scala +++ b/src/main/scala/shark/util/QueryRewriteUtils.scala @@ -24,9 +24,7 @@ import shark.memstore2.SharkTblProperties object QueryRewriteUtils { - def cacheToAlterTable( - cmd: String, - storageLevelStr: String = SharkTblProperties.STORAGE_LEVEL.defaultVal): String = { + def cacheToAlterTable(cmd: String): String = { val cmdSplit = cmd.split(' ') if (cmdSplit.size == 2) { val tableName = cmdSplit(1) @@ -34,9 +32,8 @@ object QueryRewriteUtils { ALTER TABLE %s SET TBLPROPERTIES ( 'shark.cache' = 'true', 'shark.cache.unifyView' = 'true', - 'shark.cache.reloadOnRestart' = 'true', - 'shark.cache.storageLevel' = '%s') - """.format(tableName, storageLevelStr) + 'shark.cache.reloadOnRestart' = 'true') + """.format(tableName) } else { throw new SemanticException("CACHE accepts a single table name: 'CACHE
'") } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 1b68b1b8..3734e506 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -1141,7 +1141,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val cachedCount = sc.sql("select count(*) from %s".format(tableName))(0) cachedTableCounts(i) = cachedCount } - sharkMetastore.processTablesOnShutdown() + sharkMetastore.shutdown() for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) From a945a60ea323bc4a9569c8eb5a437b762d52b98b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Nov 2013 18:27:57 -0800 Subject: [PATCH 258/331] Enable task cancellation for join outputs. --- src/main/scala/shark/execution/CoGroupedRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/CoGroupedRDD.scala b/src/main/scala/shark/execution/CoGroupedRDD.scala index 4ead29e4..79c5a30a 100644 --- a/src/main/scala/shark/execution/CoGroupedRDD.scala +++ b/src/main/scala/shark/execution/CoGroupedRDD.scala @@ -129,7 +129,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) .foreach(mergePair) } } - map.iterator + new InterruptibleIterator(context, map.iterator) } override def clearDependencies() { From 6071a8b9f0075d08df02c5fefab0874832ee1273 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Mon, 18 Nov 2013 23:31:10 -0800 Subject: [PATCH 259/331] Adding comments --- src/main/scala/shark/SharkServer.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index d4d415e2..f32eeec3 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -96,10 +96,11 @@ object SharkServer extends LogHelper { } logInfo("Audit Log: Connection Initiated with JDBC client - " + remoteClient) - val sessionID = remoteClient + '/' + System.currentTimeMillis() - // Add and enable watcher thread val jdbcSocket = t.asInstanceOf[TSocket].getSocket() jdbcSocket.setKeepAlive(true) + val sessionID = remoteClient + "/" + System.currentTimeMillis() + + // Add and enable watcher thread val watcher = new JDBCWatcher(jdbcSocket, sessionID) SharkEnv.activeSessions.add(sessionID) watcher.start() @@ -166,6 +167,10 @@ object SharkServer extends LogHelper { } } + // Detecting socket connection drops relies on TCP keep alives + // The approach is very platform specific on the duration and nature of detection + // Since java does not expose any mechanisms for tuning keepalive configurations, + // the users should explore the server OS settings for the same. class JDBCWatcher(sock:java.net.Socket, sessionID:String) extends Thread { override def run() { @@ -179,7 +184,8 @@ object SharkServer extends LogHelper { case ioe: IOException => Unit } - + // Session is terminated either manually or automatically + // clean up the jobs associated with the session ID logInfo("Session Socket connection lost, cleaning up - " + sessionID) SharkEnv.sc.cancelJobGroup(sessionID) } From 236b4cbe8eb47f68d34d58163e0329d45a82d3c0 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Mon, 18 Nov 2013 23:42:35 -0800 Subject: [PATCH 260/331] use sessionID as remoteAddress + remotePort --- src/main/scala/shark/SharkServer.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index f32eeec3..c3b2375e 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -98,8 +98,9 @@ object SharkServer extends LogHelper { val jdbcSocket = t.asInstanceOf[TSocket].getSocket() jdbcSocket.setKeepAlive(true) - val sessionID = remoteClient + "/" + System.currentTimeMillis() - + val sessionID = remoteClient + "/" + jdbcSocket + .getRemoteSocketAddress().asInstanceOf[InetSocketAddress].getPort().toString + // Add and enable watcher thread val watcher = new JDBCWatcher(jdbcSocket, sessionID) SharkEnv.activeSessions.add(sessionID) From 86a661335ce71ee7cb6b8105268ce12c009ee729 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Wed, 20 Nov 2013 16:08:41 -0800 Subject: [PATCH 261/331] Pull the TSocket information into protected if Add random number to session ID to protect temporal reuse of sockets on client --- src/main/scala/shark/SharkServer.scala | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index c3b2375e..adb7124c 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -89,19 +89,26 @@ object SharkServer extends LogHelper { val hfactory = new ThriftHiveProcessorFactory(null, new HiveConf()) { override def getProcessor(t: TTransport) = { var remoteClient = "Unknown" + + // Seed session ID by a random number + var sessionID = scala.Math.round(scala.Math.random * 10000000).toString + var jdbcSocket: java.net.Socket = null if (t.isInstanceOf[TSocket]) { remoteClient = t.asInstanceOf[TSocket].getSocket() .getRemoteSocketAddress().asInstanceOf[InetSocketAddress] .getAddress().toString() + + jdbcSocket = t.asInstanceOf[TSocket].getSocket() + jdbcSocket.setKeepAlive(true) + sessionID = remoteClient + "/" + jdbcSocket + .getRemoteSocketAddress().asInstanceOf[InetSocketAddress].getPort().toString + + "/" + sessionID + } logInfo("Audit Log: Connection Initiated with JDBC client - " + remoteClient) - val jdbcSocket = t.asInstanceOf[TSocket].getSocket() - jdbcSocket.setKeepAlive(true) - val sessionID = remoteClient + "/" + jdbcSocket - .getRemoteSocketAddress().asInstanceOf[InetSocketAddress].getPort().toString - // Add and enable watcher thread + // This handles both manual killing of session as well as connection drops val watcher = new JDBCWatcher(jdbcSocket, sessionID) SharkEnv.activeSessions.add(sessionID) watcher.start() @@ -176,8 +183,9 @@ object SharkServer extends LogHelper { override def run() { try { - while (sock.isConnected && SharkEnv.activeSessions.contains(sessionID)) { - sock.getOutputStream().write((new Array[Byte](0)).toArray) + while ((sock == null || sock.isConnected) && SharkEnv.activeSessions.contains(sessionID)) { + if (sock != null) + sock.getOutputStream().write((new Array[Byte](0)).toArray) logDebug("Session Socket Alive - " + sessionID) Thread.sleep(2*1000) } From cec4dac7d52b2de9caa1839b9d8eafac5f9be338 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sun, 27 Oct 2013 18:03:37 -0700 Subject: [PATCH 262/331] Tuples for SQL/Table 2 RDD convertors, with Python script to generate them --- .../resources/tablerdd/RDDTable_generator.py | 95 +++ .../tablerdd/SharkContext_sqlRdd_generator.py | 24 + .../tablerdd/TableRDDGenerated_generator.py | 89 +++ .../resources/tablerdd/generator_utils.py | 18 + src/main/scala/shark/SharkContext.scala | 173 +++++ src/main/scala/shark/api/RDDTable.scala | 373 +++++++--- .../scala/shark/api/RDDTableFunctions.scala | 4 +- src/main/scala/shark/api/Row.scala | 4 + .../scala/shark/api/TableRDDGenerated.scala | 638 ++++++++++++++++++ 9 files changed, 1317 insertions(+), 101 deletions(-) create mode 100755 src/main/resources/tablerdd/RDDTable_generator.py create mode 100755 src/main/resources/tablerdd/SharkContext_sqlRdd_generator.py create mode 100755 src/main/resources/tablerdd/TableRDDGenerated_generator.py create mode 100644 src/main/resources/tablerdd/generator_utils.py create mode 100644 src/main/scala/shark/api/TableRDDGenerated.scala diff --git a/src/main/resources/tablerdd/RDDTable_generator.py b/src/main/resources/tablerdd/RDDTable_generator.py new file mode 100755 index 00000000..8b359d82 --- /dev/null +++ b/src/main/resources/tablerdd/RDDTable_generator.py @@ -0,0 +1,95 @@ +#!/usr/bin/python +from string import Template +import sys +from generator_utils import * + +## This script generates RDDtable.scala + +p = sys.stdout + +# e.g. createList(1,3, "T[", "]", ",") gives T[1],T[2],T[3] +def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, indent = 0): + res = "" + oneLine = res + for y in range(start,stop+1): + res += prefix + str(y) + suffix + oneLine += prefix + str(y) + suffix + if y != stop: + res += sep + oneLine += sep + if len(oneLine) > newlineAfter: + res += "\n" + " "*indent + oneLine = "" + return res + +### The SparkContext declaration + +prefix = """ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +// *** This file is auto-generated from RDDTable_generator.py *** + +import org.apache.spark.rdd.RDD + +object RDDTableImplicits { + private type M[T] = ClassManifest[T] + +""" + +p.write(prefix) + +for x in range(2,23): + + tableClass = Template( +""" + implicit def rddToTable$num[$tmlist] + (rdd: RDD[($tlist)]): RDDTableFunctions = RDDTable(rdd) + +""").substitute(num = x, tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) + p.write(tableClass) + +prefix = """ +} + +object RDDTable { + + private type M[T] = ClassManifest[T] + private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) +""" + +p.write(prefix) + +for x in range(2,23): + + tableClass = Template( +""" + def apply[$tmlist] + (rdd: RDD[($tlist)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq($mtlist)) + } + +""").substitute(tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4), + mtlist = createList(1, x, "m[T", "]", ", ", indent=4)) + p.write(tableClass) + + +p.write("}\n") diff --git a/src/main/resources/tablerdd/SharkContext_sqlRdd_generator.py b/src/main/resources/tablerdd/SharkContext_sqlRdd_generator.py new file mode 100755 index 00000000..0f33ca71 --- /dev/null +++ b/src/main/resources/tablerdd/SharkContext_sqlRdd_generator.py @@ -0,0 +1,24 @@ +#!/usr/bin/python +from string import Template +import sys + +from generator_utils import * + +## This script generates functions sqlRdd for SharkContext.scala + +p = sys.stdout + +# The SharkContext declarations +for x in range(2,23): + sqlRddFun = Template( +""" + def sqlRdd[$list1](cmd: String): + RDD[Tuple$num[$list2]] = { + new TableRDD$num[$list2](sql2rdd(cmd), + Seq($list3)) + } +""").substitute(num = x, + list1 = createList(1, x, "T", ": M", ", ", 80, 4), + list2 = createList(1, x, "T", sep=", ", indent = 4), + list3 = createList(1, x, "m[T", "]", sep=", ", indent = 10)) + p.write(sqlRddFun) diff --git a/src/main/resources/tablerdd/TableRDDGenerated_generator.py b/src/main/resources/tablerdd/TableRDDGenerated_generator.py new file mode 100755 index 00000000..5744dea7 --- /dev/null +++ b/src/main/resources/tablerdd/TableRDDGenerated_generator.py @@ -0,0 +1,89 @@ +#!/usr/bin/python +from string import Template +import sys +from generator_utils import * + +## This script generates TableRDDGenerated.scala + +p = sys.stdout + +p.write( +""" +/* + * Copyright (C) 2013 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +// *** This file is auto-generated from TableRDDGenerated_generator.py *** + +import org.apache.spark.rdd.RDD +import org.apache.spark.{TaskContext, Partition} + +class TableSeqRDD(prev: TableRDD) + extends RDD[Seq[Any]](prev) { + + def getSchema = prev.schema + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): Iterator[Seq[Any]] = { + prev.compute(split, context).map( row => + (0 until prev.schema.size).map(i => row.getPrimitive(i)) ) + } +} + +""") + +for x in range(1,23): + + inner = "" + for y in range(1,x+1): + if y % 3 == 1: inner += " " + inner += Template(" row.getPrimitiveGeneric[T$num1]($num2)").substitute(num1=y, num2=y-1) + if y != x: inner += "," + if y % 3 == 0: inner += "\n" + inner += " ) )\n" + + tableClass = Template( +""" +class TableRDD$num[$list](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple$num[$list]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == $num, "Table only has " + tableCols + " columns, expecting $num") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple$num[$list]] = { + prev.compute(split, context).map( row => + new Tuple$num[$list]( + $innerfatlist + } +} +""").substitute(num = x, list = createList(1, x, "T", "", ", ", indent=4), innerfatlist = inner) + + + p.write(tableClass) diff --git a/src/main/resources/tablerdd/generator_utils.py b/src/main/resources/tablerdd/generator_utils.py new file mode 100644 index 00000000..26cdb487 --- /dev/null +++ b/src/main/resources/tablerdd/generator_utils.py @@ -0,0 +1,18 @@ +#!/usr/bin/python +import sys + +# e.g. createList(1,3, "T[", "]", ",") gives T[1],T[2],T[3] +def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, indent = 0): + res = "" + oneLine = res + for y in range(start,stop+1): + res += prefix + str(y) + suffix + oneLine += prefix + str(y) + suffix + if y != stop: + res += sep + oneLine += sep + if len(oneLine) > newlineAfter: + res += "\n" + " "*indent + oneLine = "" + return res + diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 0f670845..fe62122b 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.{SparkContext, SparkEnv} import shark.api._ +import org.apache.spark.rdd.RDD class SharkContext( @@ -48,6 +49,10 @@ class SharkContext( SharkContext.init() import SharkContext._ + private type M[T] = ClassManifest[T] + private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + + /** * Execute the command and return the results as a sequence. Each element * in the sequence is one row. @@ -103,6 +108,174 @@ class SharkContext( } } + /** + * Execute a SQL command and return the results as a RDD of Seq. The SQL command must be + * a SELECT statement. This is useful if the table has more than 22 columns (more than fits in tuples) + * NB: These are auto-generated using resources/tablerdd/SharkContext_sqlRdd_generator.py + */ + def sqlSeqRdd(cmd: String): RDD[Seq[Any]] = { + new TableSeqRDD(sql2rdd(cmd)) + } + + /** + * Execute a SQL command and return the results as a RDD of Tuple. The SQL command must be + * a SELECT statement. + */ + + def sqlRdd[T1: M, T2: M](cmd: String): + RDD[Tuple2[T1, T2]] = { + new TableRDD2[T1, T2](sql2rdd(cmd), + Seq(m[T1], m[T2])) + } + + def sqlRdd[T1: M, T2: M, T3: M](cmd: String): + RDD[Tuple3[T1, T2, T3]] = { + new TableRDD3[T1, T2, T3](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M](cmd: String): + RDD[Tuple4[T1, T2, T3, T4]] = { + new TableRDD4[T1, T2, T3, T4](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M](cmd: String): + RDD[Tuple5[T1, T2, T3, T4, T5]] = { + new TableRDD5[T1, T2, T3, T4, T5](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](cmd: String): + RDD[Tuple6[T1, T2, T3, T4, T5, T6]] = { + new TableRDD6[T1, T2, T3, T4, T5, T6](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M](cmd: String): + RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { + new TableRDD7[T1, T2, T3, T4, T5, T6, T7](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M](cmd: String): + RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { + new TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M](cmd: String): + RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { + new TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M](cmd: String): + RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { + new TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M](cmd: String): + RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { + new TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M](cmd: String): + RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { + new TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M](cmd: String): + RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { + new TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M](cmd: String): + RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { + new TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M](cmd: String): + RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { + new TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M](cmd: String): + RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { + new TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M](cmd: String): + RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { + new TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M](cmd: String): + RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { + new TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M](cmd: String): + RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]] = { + new TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M](cmd: String): + RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]] = { + new TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M](cmd: String): + RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]] = { + new TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) + } + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M](cmd: String): + RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]] = { + new TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22](sql2rdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) + } + /** * Execute a SQL command and collect the results locally. * diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 595ea31f..e76ea0f8 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -17,152 +17,327 @@ package shark.api +// *** This file is auto-generated from rddtable_generator.py *** + import org.apache.spark.rdd.RDD +object RDDTableImplicits { + private type M[T] = ClassManifest[T] + + + implicit def rddToTable2[T1: M, T2: M] + (rdd: RDD[(T1, T2)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable3[T1: M, T2: M, T3: M] + (rdd: RDD[(T1, T2, T3)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable4[T1: M, T2: M, T3: M, T4: M] + (rdd: RDD[(T1, T2, T3, T4)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable5[T1: M, T2: M, T3: M, T4: M, T5: M] + (rdd: RDD[(T1, T2, T3, T4, T5)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable6[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable7[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable8[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable9[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable10[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable11[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable12[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable13[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable14[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable15[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable16[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable17[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable18[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable19[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable20[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable21[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable22[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M, T22: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22)]): RDDTableFunctions = RDDTable(rdd) + + +} object RDDTable { private type M[T] = ClassManifest[T] private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) - def apply[T1: M, T2: M](rdd: RDD[(T1, T2)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2])) + def apply[T1: M, T2: M] + (rdd: RDD[(T1, T2)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2])) } - def apply[T1: M, T2: M, T3: M](rdd: RDD[(T1, T2, T3)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3])) + + def apply[T1: M, T2: M, T3: M] + (rdd: RDD[(T1, T2, T3)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3])) + } + + + def apply[T1: M, T2: M, T3: M, T4: M] + (rdd: RDD[(T1, T2, T3, T4)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4])) } - def apply[T1: M, T2: M, T3: M, T4: M](rdd: RDD[(T1, T2, T3, T4)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M] + (rdd: RDD[(T1, T2, T3, T4, T5)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M](rdd: RDD[(T1, T2, T3, T4, T5)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](rdd: RDD[(T1, T2, T3, T4, T5, T6)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M](rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13])) + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M](rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, - T18)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) - } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], + m[T20])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], + m[T20], m[T21])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M, T22: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], + m[T20], m[T21], m[T22])) } + } diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 43c686a6..cf54b08f 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -28,7 +28,7 @@ import shark.memstore2.{CacheType, TablePartitionStats, TablePartition, TablePar import shark.util.HiveUtils -class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { +class RDDTableFunctions(self: RDD[Seq[_]], manifests: Seq[ClassManifest[_]]) { def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { require(fields.size == this.manifests.size, @@ -47,7 +47,7 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { for (p <- iter) { builder.incrementRowCount() // TODO: this is not the most efficient code to do the insertion ... - p.productIterator.zipWithIndex.foreach { case (v, i) => + p.zipWithIndex.foreach { case (v, i) => builder.append(i, v.asInstanceOf[Object], ois(i)) } } diff --git a/src/main/scala/shark/api/Row.scala b/src/main/scala/shark/api/Row.scala index 1c1c2c91..96128ad8 100644 --- a/src/main/scala/shark/api/Row.scala +++ b/src/main/scala/shark/api/Row.scala @@ -97,6 +97,10 @@ class Row(val rawdata: Any, val colname2indexMap: Map[String, Int], val oi: Stru ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) } + def getPrimitiveGeneric[T](field: Int): T = getPrimitive(field).asInstanceOf[T] + + def getPrimitiveGeneric[T](field: String): T = getPrimitiveGeneric[T](colname2indexMap(field)) + ///////////////////////////////////////////////////////////////////////////////////////////////// // Complex data types // rxin: I am not sure how useful these APIs are since they would expose the Hive internal diff --git a/src/main/scala/shark/api/TableRDDGenerated.scala b/src/main/scala/shark/api/TableRDDGenerated.scala new file mode 100644 index 00000000..344ee6a7 --- /dev/null +++ b/src/main/scala/shark/api/TableRDDGenerated.scala @@ -0,0 +1,638 @@ +/* + * Copyright (C) 2013 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +// *** This file is auto-generated from TableRDDGenerated_generator.py *** + +import org.apache.spark.rdd.RDD +import org.apache.spark.{TaskContext, Partition} + +class TableSeqRDD(prev: TableRDD) + extends RDD[Seq[Any]](prev) { + + def getSchema = prev.schema + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): Iterator[Seq[Any]] = { + prev.compute(split, context).map( row => + (0 until prev.schema.size).map(i => row.getPrimitive(i)) ) + } +} + + +class TableRDD1[T1](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple1[T1]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 1, "Table only has " + tableCols + " columns, expecting 1") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple1[T1]] = { + prev.compute(split, context).map( row => + new Tuple1[T1]( + row.getPrimitiveGeneric[T1](0) ) ) + + } +} + +class TableRDD2[T1, T2](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple2[T1, T2]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 2, "Table only has " + tableCols + " columns, expecting 2") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple2[T1, T2]] = { + prev.compute(split, context).map( row => + new Tuple2[T1, T2]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) + + } +} + +class TableRDD3[T1, T2, T3](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple3[T1, T2, T3]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 3, "Table only has " + tableCols + " columns, expecting 3") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple3[T1, T2, T3]] = { + prev.compute(split, context).map( row => + new Tuple3[T1, T2, T3]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) + ) ) + + } +} + +class TableRDD4[T1, T2, T3, T4](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple4[T1, T2, T3, T4]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 4, "Table only has " + tableCols + " columns, expecting 4") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple4[T1, T2, T3, T4]] = { + prev.compute(split, context).map( row => + new Tuple4[T1, T2, T3, T4]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3) ) ) + + } +} + +class TableRDD5[T1, T2, T3, T4, T5](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple5[T1, T2, T3, T4, T5]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 5, "Table only has " + tableCols + " columns, expecting 5") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple5[T1, T2, T3, T4, T5]] = { + prev.compute(split, context).map( row => + new Tuple5[T1, T2, T3, T4, T5]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4) ) ) + + } +} + +class TableRDD6[T1, T2, T3, T4, T5, T6](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple6[T1, T2, T3, T4, T5, T6]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 6, "Table only has " + tableCols + " columns, expecting 6") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple6[T1, T2, T3, T4, T5, T6]] = { + prev.compute(split, context).map( row => + new Tuple6[T1, T2, T3, T4, T5, T6]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5) + ) ) + + } +} + +class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 7, "Table only has " + tableCols + " columns, expecting 7") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { + prev.compute(split, context).map( row => + new Tuple7[T1, T2, T3, T4, T5, T6, T7]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6) ) ) + + } +} + +class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 8, "Table only has " + tableCols + " columns, expecting 8") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { + prev.compute(split, context).map( row => + new Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7) ) ) + + } +} + +class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 9, "Table only has " + tableCols + " columns, expecting 9") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { + prev.compute(split, context).map( row => + new Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8) + ) ) + + } +} + +class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 10, "Table only has " + tableCols + " columns, expecting 10") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { + prev.compute(split, context).map( row => + new Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9) ) ) + + } +} + +class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 11, "Table only has " + tableCols + " columns, expecting 11") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { + prev.compute(split, context).map( row => + new Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10) ) ) + + } +} + +class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 12, "Table only has " + tableCols + " columns, expecting 12") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { + prev.compute(split, context).map( row => + new Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11) + ) ) + + } +} + +class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 13, "Table only has " + tableCols + " columns, expecting 13") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { + prev.compute(split, context).map( row => + new Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12) ) ) + + } +} + +class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 14, "Table only has " + tableCols + " columns, expecting 14") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { + prev.compute(split, context).map( row => + new Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13) ) ) + + } +} + +class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 15, "Table only has " + tableCols + " columns, expecting 15") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { + prev.compute(split, context).map( row => + new Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14) + ) ) + + } +} + +class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 16, "Table only has " + tableCols + " columns, expecting 16") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { + prev.compute(split, context).map( row => + new Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15) ) ) + + } +} + +class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 17, "Table only has " + tableCols + " columns, expecting 17") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { + prev.compute(split, context).map( row => + new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16) ) ) + + } +} + +class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 18, "Table only has " + tableCols + " columns, expecting 18") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { + prev.compute(split, context).map( row => + new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17) + ) ) + + } +} + +class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 19, "Table only has " + tableCols + " columns, expecting 19") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]] = { + prev.compute(split, context).map( row => + new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18) ) ) + + } +} + +class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 20, "Table only has " + tableCols + " columns, expecting 20") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]] = { + prev.compute(split, context).map( row => + new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19) ) ) + + } +} + +class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 21, "Table only has " + tableCols + " columns, expecting 21") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]] = { + prev.compute(split, context).map( row => + new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20) + ) ) + + } +} + +class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21, T22](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]](prev) { + def schema = prev.schema + + private val tableCols = schema.size + require(tableCols == 22, "Table only has " + tableCols + " columns, expecting 22") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]] = { + prev.compute(split, context).map( row => + new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20), + row.getPrimitiveGeneric[T22](21) ) ) + + } +} From e7bfe4d2733d4e5511d707bc99e979326b1e0cd5 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sun, 27 Oct 2013 18:03:37 -0700 Subject: [PATCH 263/331] Tuples for SQL/Table 2 RDD convertors, with Python script to generate them --- .../tablerdd/table_rdd_generators.py | 75 ++ src/main/scala/shark/SharkContext.scala | 155 ++++- src/main/scala/shark/SharkDriver.scala | 4 +- .../scala/shark/api/JavaSharkContext.scala | 2 +- src/main/scala/shark/api/RDDTable.scala | 4 + src/main/scala/shark/api/Row.scala | 4 + src/main/scala/shark/api/RowRDD.scala | 80 +++ src/main/scala/shark/api/TableRDD.scala | 638 ++++++++++++++++-- .../scala/shark/execution/SparkTask.scala | 8 +- src/test/scala/shark/SQLSuite.scala | 8 +- 10 files changed, 922 insertions(+), 56 deletions(-) create mode 100755 src/main/resources/tablerdd/table_rdd_generators.py create mode 100644 src/main/scala/shark/api/RowRDD.scala diff --git a/src/main/resources/tablerdd/table_rdd_generators.py b/src/main/resources/tablerdd/table_rdd_generators.py new file mode 100755 index 00000000..8946c706 --- /dev/null +++ b/src/main/resources/tablerdd/table_rdd_generators.py @@ -0,0 +1,75 @@ +#!/usr/bin/python +from string import Template +import sys + +## This script generates the 22 functions needed to create sqlToRdd's + +p = sys.stdout + +# e.g. createList(1,3, "T[", "]", ",") gives T[1],T[2],T[3] +def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 80, indent = 0): + res = "" + oneLine = res + for y in range(start,stop+1): + res += prefix + str(y) + suffix + oneLine += prefix + str(y) + suffix + if y != stop: + res += sep + oneLine += sep + if len(oneLine) > newlineAfter: + res += "\n" + " "*indent + oneLine = "" + return res + +### The SparkContext declaration +for x in range(2,23): + + inner = "" + for y in range(1,x+1): + if y % 3 == 1: inner += " " + inner += Template(" row.getPrimitiveGeneric[T$num1]($num2)").substitute(num1=y, num2=y-1) + if y != x: inner += "," + if y % 3 == 0: inner += "\n" + inner += " ) )\n" + + tableClass = Template( +""" +class TableRDD$num[$list](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple$num[$list]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != $num) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting $num") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple$num[$list]] = { + prev.compute(split, context).map( row => + new Tuple$num[$list]( + $innerfatlist + } +} +""").substitute(num = x, list = createList(1, x, "T", "", ", ", indent=4), innerfatlist = inner) + + + p.write(tableClass) + +# The SharkContext declarations +for x in range(2,23): + sqlRddFun = Template( +""" + def sqlRdd[$list1](cmd: String): + RDD[Tuple$num[$list2]] = + new TableRDD$num[$list2](sqlRowRdd(cmd), + Seq($list3)) +""").substitute(num = x, + list1 = createList(1, x, "T", ": M", ", ", 80, 4), + list2 = createList(1, x, "T", sep=", ", indent = 4), + list3 = createList(1, x, "m[T", "]", sep=", ", indent = 10)) + p.write(sqlRddFun) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 0f670845..0143bd92 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.{SparkContext, SparkEnv} import shark.api._ +import org.apache.spark.rdd.RDD class SharkContext( @@ -48,6 +49,10 @@ class SharkContext( SharkContext.init() import SharkContext._ + private type M[T] = ClassManifest[T] + private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + + /** * Execute the command and return the results as a sequence. Each element * in the sequence is one row. @@ -91,7 +96,7 @@ class SharkContext( * Execute a SQL command and return the results as a TableRDD. The SQL command must be * a SELECT statement. */ - def sql2rdd(cmd: String): TableRDD = { + def sqlRowRdd(cmd: String): RowRDD = { SparkEnv.set(sparkEnv) SessionState.start(sessionState) val driver = new SharkDriver(hiveconf) @@ -103,6 +108,154 @@ class SharkContext( } } + /** + * Execute a SQL command and return the results as a RDD of Seq. The SQL command must be + * a SELECT statement. This is useful if the table has more than 22 columns (more than fits in tuples) + * NB: These are auto-generated using resources/tablerdd/table_rdd_generators.py + */ + def sqlSeqRdd(cmd: String): RDD[Seq[Any]] = + new TableSeqRDD(sqlRowRdd(cmd)) + + /** + * Execute a SQL command and return the results as a RDD of Tuple. The SQL command must be + * a SELECT statement. + */ + def sqlRdd[T](cmd: String): RDD[Tuple1[T]] = + new TableRDD1[T](sqlRowRdd(cmd)) + + def sqlRdd[T1: M, T2: M](cmd: String): + RDD[Tuple2[T1, T2]] = + new TableRDD2[T1, T2](sqlRowRdd(cmd), + Seq(m[T1], m[T2])) + + def sqlRdd[T1: M, T2: M, T3: M](cmd: String): + RDD[Tuple3[T1, T2, T3]] = + new TableRDD3[T1, T2, T3](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M](cmd: String): + RDD[Tuple4[T1, T2, T3, T4]] = + new TableRDD4[T1, T2, T3, T4](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M](cmd: String): + RDD[Tuple5[T1, T2, T3, T4, T5]] = + new TableRDD5[T1, T2, T3, T4, T5](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](cmd: String): + RDD[Tuple6[T1, T2, T3, T4, T5, T6]] = + new TableRDD6[T1, T2, T3, T4, T5, T6](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M](cmd: String): + RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = + new TableRDD7[T1, T2, T3, T4, T5, T6, T7](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M](cmd: String): + RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = + new TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M](cmd: String): + RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = + new TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M](cmd: String): + RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = + new TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M](cmd: String): + RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = + new TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M](cmd: String): + RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = + new TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M](cmd: String): + RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = + new TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M](cmd: String): + RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = + new TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M](cmd: String): + RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = + new TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M](cmd: String): + RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = + new TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M](cmd: String): + RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = + new TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M](cmd: String): + RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = + new TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M](cmd: String): + RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]] = + new TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M](cmd: String): + RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]] = + new TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M](cmd: String): + RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]] = + new TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) + + def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M](cmd: String): + RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]] = + new TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22](sqlRowRdd(cmd), + Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], + m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) + /** * Execute a SQL command and collect the results locally. * diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 1869c946..1812cde4 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.{SerDe, SerDeUtils} import org.apache.hadoop.util.StringUtils -import shark.api.TableRDD +import shark.api.RowRDD import shark.api.QueryExecutionException import shark.execution.{SharkDDLTask, SharkDDLWork, SharkExplainTask, SharkExplainWork, SparkTask, SparkWork} @@ -143,7 +143,7 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe super.init() } - def tableRdd(cmd: String): Option[TableRDD] = { + def tableRdd(cmd: String): Option[RowRDD] = { useTableRddSink = true val response = run(cmd) // Throw an exception if there is an error in query processing. diff --git a/src/main/scala/shark/api/JavaSharkContext.scala b/src/main/scala/shark/api/JavaSharkContext.scala index 9ec2914a..53d47447 100644 --- a/src/main/scala/shark/api/JavaSharkContext.scala +++ b/src/main/scala/shark/api/JavaSharkContext.scala @@ -78,7 +78,7 @@ class JavaSharkContext(val sharkCtx: SharkContext) extends JavaSparkContext(shar * Execute the command and return the results as a TableRDD. */ def sql2rdd(cmd: String): JavaTableRDD = { - val rdd = sharkCtx.sql2rdd(cmd) + val rdd = sharkCtx.sqlRowRdd(cmd) new JavaTableRDD(rdd, rdd.schema) } diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 595ea31f..d93fac1d 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -25,6 +25,10 @@ object RDDTable { private type M[T] = ClassManifest[T] private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + /* + * NB: These functions are auto-generated using resources/tablerdd/table_rdd_generators.py + */ + def apply[T1: M, T2: M](rdd: RDD[(T1, T2)]) = { new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2])) } diff --git a/src/main/scala/shark/api/Row.scala b/src/main/scala/shark/api/Row.scala index e0f34869..f91cd05d 100644 --- a/src/main/scala/shark/api/Row.scala +++ b/src/main/scala/shark/api/Row.scala @@ -110,6 +110,10 @@ class Row(val rawdata: Any, val colname2indexMap: Map[String, Int], val oi: Stru ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) } + def getPrimitiveGeneric[T](field: Int): T = getPrimitive(field).asInstanceOf[T] + + def getPrimitiveGeneric[T](field: String): T = getPrimitiveGeneric[T](colname2indexMap(field)) + ///////////////////////////////////////////////////////////////////////////////////////////////// // Complex data types - only return the string representation of them for now. ///////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/src/main/scala/shark/api/RowRDD.scala b/src/main/scala/shark/api/RowRDD.scala new file mode 100644 index 00000000..b09b484f --- /dev/null +++ b/src/main/scala/shark/api/RowRDD.scala @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +import java.util.{List => JList} + +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector, StructObjectInspector} + +import shark.execution.serialization.KryoSerializer + +import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.rdd.RDD + +class RowRDD( + prev: RDD[Any], + val schema: Array[ColumnDesc], + @transient oi: ObjectInspector, + val limit: Int = -1) + extends RDD[Row](prev) { + + private[shark] + def this(prev: RDD[Any], schema: JList[FieldSchema], oi: ObjectInspector, limit: Int) { + this(prev, ColumnDesc.createSchema(schema), oi, limit) + } + + override def getPartitions = firstParent[Any].partitions + + override def compute(split: Partition, context: TaskContext): Iterator[Row] = { + val structOi = initObjectInspector() + firstParent[Any].iterator(split, context).map { rowData => + new Row(rowData, colname2indexMap, structOi) + } + } + + /** + * ObjectInspector is not Java serializable. We serialize it using Kryo and + * and save it as a byte array. On slave nodes, we deserialize this byte + * array to obtain the ObjectInspector object. + */ + private val serializedObjectInspector: Array[Byte] = KryoSerializer.serialize(oi) + + /** + * Maps the column name to column index. + */ + private val colname2indexMap: Map[String, Int] = + collection.immutable.Map() ++ schema.zipWithIndex.map { case(column, index) => + (column.name, index) + } + + /** + * Initialize object inspector from the serializedObjectInspector. + */ + private def initObjectInspector(): StructObjectInspector = { + val oi = KryoSerializer.deserialize[ObjectInspector](serializedObjectInspector) + oi match { + case soi: StructObjectInspector => soi + case _ => throw new Exception("Only basic StructObjectInspector is supposed.") + } + } + +} + + + diff --git a/src/main/scala/shark/api/TableRDD.scala b/src/main/scala/shark/api/TableRDD.scala index 75929f80..c6a433bb 100644 --- a/src/main/scala/shark/api/TableRDD.scala +++ b/src/main/scala/shark/api/TableRDD.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2012 The Regents of The University California. + * Copyright (C) 2013 The Regents of The University California. * All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -17,64 +17,614 @@ package shark.api -import java.util.{List => JList} +import org.apache.spark.rdd.RDD +import org.apache.spark.{TaskContext, Partition} -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} +class TableSeqRDD(prev: RowRDD) + extends RDD[Seq[Any]](prev) { -import shark.execution.serialization.KryoSerializer + def getSchema = prev.schema -import org.apache.spark.{Partition, TaskContext} -import org.apache.spark.rdd.RDD + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Seq[Any]] = { + prev.compute(split, context).map( row => + (0 until prev.schema.size).map(i => row.getPrimitive(i)) ) + } +} + +class TableRDD1[T1](prev: RowRDD) + extends RDD[Tuple1[T1]](prev) { + + def getSchema = prev.schema + + val tableCols = prev.schema.size + if (tableCols != 1) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 1") + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple1[T1]] = { + prev.compute(split, context).map( row => + new Tuple1[T1]( + row.getPrimitiveGeneric[T1](0) + ) ) + } +} + +class TableRDD2[T1, T2](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple2[T1, T2]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 2) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 2") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple2[T1, T2]] = { + prev.compute(split, context).map( row => + new Tuple2[T1, T2]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) + + } +} + +class TableRDD3[T1, T2, T3](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple3[T1, T2, T3]](prev) { + def schema = prev.schema + val tableCols = schema.size + if (tableCols != 3) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 3") -class TableRDD( - prev: RDD[Any], - val schema: Array[ColumnDesc], - @transient oi: ObjectInspector, - val limit: Int = -1) - extends RDD[Row](prev) { + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple3[T1, T2, T3]] = { + prev.compute(split, context).map( row => + new Tuple3[T1, T2, T3]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) + ) ) - private[shark] - def this(prev: RDD[Any], schema: JList[FieldSchema], oi: ObjectInspector, limit: Int) { - this(prev, ColumnDesc.createSchema(schema), oi, limit) } +} + +class TableRDD4[T1, T2, T3, T4](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple4[T1, T2, T3, T4]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 4) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 4") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - override def getPartitions = firstParent[Any].partitions + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple4[T1, T2, T3, T4]] = { + prev.compute(split, context).map( row => + new Tuple4[T1, T2, T3, T4]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3) ) ) - override def compute(split: Partition, context: TaskContext): Iterator[Row] = { - val structOi = initObjectInspector() - firstParent[Any].iterator(split, context).map { rowData => - new Row(rowData, colname2indexMap, structOi) - } } +} + +class TableRDD5[T1, T2, T3, T4, T5](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple5[T1, T2, T3, T4, T5]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 5) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 5") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - /** - * ObjectInspector is not Java serializable. We serialize it using Kryo and - * and save it as a byte array. On slave nodes, we deserialize this byte - * array to obtain the ObjectInspector object. - */ - private val serializedObjectInspector: Array[Byte] = KryoSerializer.serialize(oi) + override def getPartitions = prev.getPartitions - /** - * Maps the column name to column index. - */ - private val colname2indexMap: Map[String, Int] = - collection.immutable.Map() ++ schema.zipWithIndex.map { case(column, index) => - (column.name, index) - } + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple5[T1, T2, T3, T4, T5]] = { + prev.compute(split, context).map( row => + new Tuple5[T1, T2, T3, T4, T5]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4) ) ) - /** - * Initialize object inspector from the serializedObjectInspector. - */ - private def initObjectInspector(): StructObjectInspector = { - val oi = KryoSerializer.deserialize[ObjectInspector](serializedObjectInspector) - oi match { - case soi: StructObjectInspector => soi - case _ => throw new Exception("Only basic StructObjectInspector is supposed.") - } } } +class TableRDD6[T1, T2, T3, T4, T5, T6](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple6[T1, T2, T3, T4, T5, T6]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 6) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 6") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple6[T1, T2, T3, T4, T5, T6]] = { + prev.compute(split, context).map( row => + new Tuple6[T1, T2, T3, T4, T5, T6]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5) + ) ) + + } +} + +class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 7) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 7") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { + prev.compute(split, context).map( row => + new Tuple7[T1, T2, T3, T4, T5, T6, T7]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6) ) ) + + } +} + +class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 8) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 8") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { + prev.compute(split, context).map( row => + new Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7) ) ) + + } +} + +class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 9) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 9") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { + prev.compute(split, context).map( row => + new Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8) + ) ) + + } +} + +class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 10) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 10") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { + prev.compute(split, context).map( row => + new Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9) ) ) + + } +} + +class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 11) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 11") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { + prev.compute(split, context).map( row => + new Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10) ) ) + + } +} + +class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 12) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 12") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { + prev.compute(split, context).map( row => + new Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11) + ) ) + } +} + +class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 13) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 13") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { + prev.compute(split, context).map( row => + new Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12) ) ) + + } +} + +class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 14) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 14") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { + prev.compute(split, context).map( row => + new Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13) ) ) + + } +} + +class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 15) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 15") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { + prev.compute(split, context).map( row => + new Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14) + ) ) + + } +} + +class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 16) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 16") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { + prev.compute(split, context).map( row => + new Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15) ) ) + + } +} + +class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 17) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 17") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { + prev.compute(split, context).map( row => + new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16) ) ) + + } +} + +class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 18) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 18") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { + prev.compute(split, context).map( row => + new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17) + ) ) + + } +} + +class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 19) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 19") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]] = { + prev.compute(split, context).map( row => + new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18) ) ) + + } +} + +class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 20) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 20") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]] = { + prev.compute(split, context).map( row => + new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19) ) ) + + } +} + +class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 21) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 21") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]] = { + prev.compute(split, context).map( row => + new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20) + ) ) + + } +} + +class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21, T22](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 22) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 22") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]] = { + prev.compute(split, context).map( row => + new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20), + row.getPrimitiveGeneric[T22](21) ) ) + + } +} diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index 04dea899..ca5dd853 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.rdd.RDD -import shark.api.TableRDD +import shark.api.RowRDD import shark.{LogHelper, SharkEnv} @@ -49,9 +49,9 @@ extends java.io.Serializable private[shark] class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { - private var _tableRdd: Option[TableRDD] = None + private var _tableRdd: Option[RowRDD] = None - def tableRdd: Option[TableRDD] = _tableRdd + def tableRdd: Option[RowRDD] = _tableRdd override def execute(driverContext: DriverContext): Int = { logDebug("Executing " + this.getClass.getName) @@ -104,7 +104,7 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { } if (terminalOp.isInstanceOf[TableRddSinkOperator]) { - _tableRdd = Some(new TableRDD(sinkRdd, work.resultSchema, terminalOp.objectInspector, limit)) + _tableRdd = Some(new RowRDD(sinkRdd, work.resultSchema, terminalOp.objectInspector, limit)) } 0 diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index d3832478..a8e5dcca 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -834,9 +834,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("sql2rdd") { - var rdd = sc.sql2rdd("select * from test") + var rdd = sc.sqlRowRdd("select * from test") assert(rdd.count === 500) - rdd = sc.sql2rdd("select * from test_cached") + rdd = sc.sqlRowRdd("select * from test_cached") assert(rdd.count === 500) val collected = rdd.map(r => r.getInt(0)).collect().sortWith(_ < _) assert(collected(0) === 0) @@ -845,7 +845,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("null values in sql2rdd") { - val nullsRdd = sc.sql2rdd("select * from test_null where key is null") + val nullsRdd = sc.sqlRowRdd("select * from test_null where key is null") val nulls = nullsRdd.map(r => r.getInt(0)).collect() assert(nulls(0) === null) assert(nulls.size === 10) @@ -857,7 +857,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("sql2rdd exception") { - val e = intercept[QueryExecutionException] { sc.sql2rdd("asdfasdfasdfasdf") } + val e = intercept[QueryExecutionException] { sc.sqlRowRdd("asdfasdfasdfasdf") } e.getMessage.contains("semantic") } } From c12f69adc3419318fc49c86ae96b0ee31ddbbfda Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 13 Nov 2013 00:15:44 -0800 Subject: [PATCH 264/331] Changing back to not break API --- src/main/scala/shark/api/RowRDD.scala | 80 --- src/main/scala/shark/api/TableRDD.scala | 638 ++---------------- .../scala/shark/api/TableRDDGenerated.scala | 630 +++++++++++++++++ 3 files changed, 674 insertions(+), 674 deletions(-) delete mode 100644 src/main/scala/shark/api/RowRDD.scala create mode 100644 src/main/scala/shark/api/TableRDDGenerated.scala diff --git a/src/main/scala/shark/api/RowRDD.scala b/src/main/scala/shark/api/RowRDD.scala deleted file mode 100644 index b09b484f..00000000 --- a/src/main/scala/shark/api/RowRDD.scala +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.api - -import java.util.{List => JList} - -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector, StructObjectInspector} - -import shark.execution.serialization.KryoSerializer - -import org.apache.spark.{Partition, TaskContext} -import org.apache.spark.rdd.RDD - -class RowRDD( - prev: RDD[Any], - val schema: Array[ColumnDesc], - @transient oi: ObjectInspector, - val limit: Int = -1) - extends RDD[Row](prev) { - - private[shark] - def this(prev: RDD[Any], schema: JList[FieldSchema], oi: ObjectInspector, limit: Int) { - this(prev, ColumnDesc.createSchema(schema), oi, limit) - } - - override def getPartitions = firstParent[Any].partitions - - override def compute(split: Partition, context: TaskContext): Iterator[Row] = { - val structOi = initObjectInspector() - firstParent[Any].iterator(split, context).map { rowData => - new Row(rowData, colname2indexMap, structOi) - } - } - - /** - * ObjectInspector is not Java serializable. We serialize it using Kryo and - * and save it as a byte array. On slave nodes, we deserialize this byte - * array to obtain the ObjectInspector object. - */ - private val serializedObjectInspector: Array[Byte] = KryoSerializer.serialize(oi) - - /** - * Maps the column name to column index. - */ - private val colname2indexMap: Map[String, Int] = - collection.immutable.Map() ++ schema.zipWithIndex.map { case(column, index) => - (column.name, index) - } - - /** - * Initialize object inspector from the serializedObjectInspector. - */ - private def initObjectInspector(): StructObjectInspector = { - val oi = KryoSerializer.deserialize[ObjectInspector](serializedObjectInspector) - oi match { - case soi: StructObjectInspector => soi - case _ => throw new Exception("Only basic StructObjectInspector is supposed.") - } - } - -} - - - diff --git a/src/main/scala/shark/api/TableRDD.scala b/src/main/scala/shark/api/TableRDD.scala index c6a433bb..b09b484f 100644 --- a/src/main/scala/shark/api/TableRDD.scala +++ b/src/main/scala/shark/api/TableRDD.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2013 The Regents of The University California. + * Copyright (C) 2012 The Regents of The University California. * All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -17,614 +17,64 @@ package shark.api -import org.apache.spark.rdd.RDD -import org.apache.spark.{TaskContext, Partition} - -class TableSeqRDD(prev: RowRDD) - extends RDD[Seq[Any]](prev) { - - def getSchema = prev.schema - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Seq[Any]] = { - prev.compute(split, context).map( row => - (0 until prev.schema.size).map(i => row.getPrimitive(i)) ) - } -} - -class TableRDD1[T1](prev: RowRDD) - extends RDD[Tuple1[T1]](prev) { - - def getSchema = prev.schema - - val tableCols = prev.schema.size - if (tableCols != 1) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 1") - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple1[T1]] = { - prev.compute(split, context).map( row => - new Tuple1[T1]( - row.getPrimitiveGeneric[T1](0) - ) ) - } -} - -class TableRDD2[T1, T2](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple2[T1, T2]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 2) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 2") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple2[T1, T2]] = { - prev.compute(split, context).map( row => - new Tuple2[T1, T2]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) - - } -} - -class TableRDD3[T1, T2, T3](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple3[T1, T2, T3]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 3) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 3") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple3[T1, T2, T3]] = { - prev.compute(split, context).map( row => - new Tuple3[T1, T2, T3]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) - ) ) - - } -} - -class TableRDD4[T1, T2, T3, T4](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple4[T1, T2, T3, T4]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 4) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 4") +import java.util.{List => JList} - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector, StructObjectInspector} - override def getPartitions = prev.getPartitions +import shark.execution.serialization.KryoSerializer - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple4[T1, T2, T3, T4]] = { - prev.compute(split, context).map( row => - new Tuple4[T1, T2, T3, T4]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3) ) ) - - } -} - -class TableRDD5[T1, T2, T3, T4, T5](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple5[T1, T2, T3, T4, T5]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 5) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 5") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple5[T1, T2, T3, T4, T5]] = { - prev.compute(split, context).map( row => - new Tuple5[T1, T2, T3, T4, T5]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4) ) ) - - } -} - -class TableRDD6[T1, T2, T3, T4, T5, T6](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple6[T1, T2, T3, T4, T5, T6]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 6) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 6") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple6[T1, T2, T3, T4, T5, T6]] = { - prev.compute(split, context).map( row => - new Tuple6[T1, T2, T3, T4, T5, T6]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5) - ) ) - - } -} - -class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 7) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 7") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { - prev.compute(split, context).map( row => - new Tuple7[T1, T2, T3, T4, T5, T6, T7]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6) ) ) - - } -} - -class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 8) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 8") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { - prev.compute(split, context).map( row => - new Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7) ) ) - - } -} - -class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 9) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 9") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { - prev.compute(split, context).map( row => - new Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8) - ) ) - - } -} - -class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 10) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 10") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { - prev.compute(split, context).map( row => - new Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9) ) ) - - } -} - -class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 11) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 11") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { - prev.compute(split, context).map( row => - new Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10) ) ) - - } -} - -class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 12) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 12") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { - prev.compute(split, context).map( row => - new Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11) - ) ) - - } -} - -class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 13) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 13") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { - prev.compute(split, context).map( row => - new Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12) ) ) - - } -} - -class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 14) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 14") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { - prev.compute(split, context).map( row => - new Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13) ) ) - - } -} - -class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 15) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 15") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { - prev.compute(split, context).map( row => - new Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14) - ) ) - - } -} - -class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 16) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 16") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions +import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.rdd.RDD - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { - prev.compute(split, context).map( row => - new Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), - row.getPrimitiveGeneric[T16](15) ) ) +class RowRDD( + prev: RDD[Any], + val schema: Array[ColumnDesc], + @transient oi: ObjectInspector, + val limit: Int = -1) + extends RDD[Row](prev) { + private[shark] + def this(prev: RDD[Any], schema: JList[FieldSchema], oi: ObjectInspector, limit: Int) { + this(prev, ColumnDesc.createSchema(schema), oi, limit) } -} - -class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { - def schema = prev.schema - val tableCols = schema.size - if (tableCols != 17) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 17") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { - prev.compute(split, context).map( row => - new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), - row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16) ) ) + override def getPartitions = firstParent[Any].partitions + override def compute(split: Partition, context: TaskContext): Iterator[Row] = { + val structOi = initObjectInspector() + firstParent[Any].iterator(split, context).map { rowData => + new Row(rowData, colname2indexMap, structOi) + } } -} - -class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { - def schema = prev.schema - val tableCols = schema.size - if (tableCols != 18) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 18") + /** + * ObjectInspector is not Java serializable. We serialize it using Kryo and + * and save it as a byte array. On slave nodes, we deserialize this byte + * array to obtain the ObjectInspector object. + */ + private val serializedObjectInspector: Array[Byte] = KryoSerializer.serialize(oi) - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { - prev.compute(split, context).map( row => - new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), - row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17) - ) ) + /** + * Maps the column name to column index. + */ + private val colname2indexMap: Map[String, Int] = + collection.immutable.Map() ++ schema.zipWithIndex.map { case(column, index) => + (column.name, index) + } + /** + * Initialize object inspector from the serializedObjectInspector. + */ + private def initObjectInspector(): StructObjectInspector = { + val oi = KryoSerializer.deserialize[ObjectInspector](serializedObjectInspector) + oi match { + case soi: StructObjectInspector => soi + case _ => throw new Exception("Only basic StructObjectInspector is supposed.") + } } -} - -class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 19) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 19") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19]] = { - prev.compute(split, context).map( row => - new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), - row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), - row.getPrimitiveGeneric[T19](18) ) ) - - } } -class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 20) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 20") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20]] = { - prev.compute(split, context).map( row => - new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), - row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), - row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19) ) ) - - } -} - -class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20, T21](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 21) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 21") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21]] = { - prev.compute(split, context).map( row => - new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), - row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), - row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20) - ) ) - - } -} - -class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20, T21, T22](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != 22) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 22") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22]] = { - prev.compute(split, context).map( row => - new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), - row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), - row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), - row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), - row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), - row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), - row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20), - row.getPrimitiveGeneric[T22](21) ) ) - - } -} diff --git a/src/main/scala/shark/api/TableRDDGenerated.scala b/src/main/scala/shark/api/TableRDDGenerated.scala new file mode 100644 index 00000000..c6a433bb --- /dev/null +++ b/src/main/scala/shark/api/TableRDDGenerated.scala @@ -0,0 +1,630 @@ +/* + * Copyright (C) 2013 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +import org.apache.spark.rdd.RDD +import org.apache.spark.{TaskContext, Partition} + +class TableSeqRDD(prev: RowRDD) + extends RDD[Seq[Any]](prev) { + + def getSchema = prev.schema + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Seq[Any]] = { + prev.compute(split, context).map( row => + (0 until prev.schema.size).map(i => row.getPrimitive(i)) ) + } +} + +class TableRDD1[T1](prev: RowRDD) + extends RDD[Tuple1[T1]](prev) { + + def getSchema = prev.schema + + val tableCols = prev.schema.size + if (tableCols != 1) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 1") + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple1[T1]] = { + prev.compute(split, context).map( row => + new Tuple1[T1]( + row.getPrimitiveGeneric[T1](0) + ) ) + } +} + +class TableRDD2[T1, T2](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple2[T1, T2]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 2) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 2") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple2[T1, T2]] = { + prev.compute(split, context).map( row => + new Tuple2[T1, T2]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) + + } +} + +class TableRDD3[T1, T2, T3](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple3[T1, T2, T3]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 3) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 3") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple3[T1, T2, T3]] = { + prev.compute(split, context).map( row => + new Tuple3[T1, T2, T3]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) + ) ) + + } +} + +class TableRDD4[T1, T2, T3, T4](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple4[T1, T2, T3, T4]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 4) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 4") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple4[T1, T2, T3, T4]] = { + prev.compute(split, context).map( row => + new Tuple4[T1, T2, T3, T4]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3) ) ) + + } +} + +class TableRDD5[T1, T2, T3, T4, T5](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple5[T1, T2, T3, T4, T5]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 5) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 5") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple5[T1, T2, T3, T4, T5]] = { + prev.compute(split, context).map( row => + new Tuple5[T1, T2, T3, T4, T5]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4) ) ) + + } +} + +class TableRDD6[T1, T2, T3, T4, T5, T6](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple6[T1, T2, T3, T4, T5, T6]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 6) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 6") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple6[T1, T2, T3, T4, T5, T6]] = { + prev.compute(split, context).map( row => + new Tuple6[T1, T2, T3, T4, T5, T6]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5) + ) ) + + } +} + +class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 7) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 7") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { + prev.compute(split, context).map( row => + new Tuple7[T1, T2, T3, T4, T5, T6, T7]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6) ) ) + + } +} + +class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 8) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 8") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { + prev.compute(split, context).map( row => + new Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7) ) ) + + } +} + +class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 9) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 9") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { + prev.compute(split, context).map( row => + new Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8) + ) ) + + } +} + +class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 10) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 10") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { + prev.compute(split, context).map( row => + new Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9) ) ) + + } +} + +class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 11) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 11") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { + prev.compute(split, context).map( row => + new Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10) ) ) + + } +} + +class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 12) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 12") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { + prev.compute(split, context).map( row => + new Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11) + ) ) + + } +} + +class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 13) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 13") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { + prev.compute(split, context).map( row => + new Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12) ) ) + + } +} + +class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 14) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 14") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { + prev.compute(split, context).map( row => + new Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13) ) ) + + } +} + +class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 15) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 15") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { + prev.compute(split, context).map( row => + new Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14) + ) ) + + } +} + +class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 16) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 16") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { + prev.compute(split, context).map( row => + new Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15) ) ) + + } +} + +class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 17) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 17") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { + prev.compute(split, context).map( row => + new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16) ) ) + + } +} + +class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 18) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 18") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { + prev.compute(split, context).map( row => + new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17) + ) ) + + } +} + +class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 19) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 19") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]] = { + prev.compute(split, context).map( row => + new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18) ) ) + + } +} + +class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 20) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 20") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]] = { + prev.compute(split, context).map( row => + new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19) ) ) + + } +} + +class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 21) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 21") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]] = { + prev.compute(split, context).map( row => + new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20) + ) ) + + } +} + +class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21, T22](prev: RowRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]](prev) { + def schema = prev.schema + + val tableCols = schema.size + if (tableCols != 22) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting 22") + + mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + throw new IllegalArgumentException( + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + + override def getPartitions = prev.getPartitions + + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]] = { + prev.compute(split, context).map( row => + new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), + row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), + row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), + row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), + row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), + row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20), + row.getPrimitiveGeneric[T22](21) ) ) + + } +} From ae310d905ddd15535d01c848d05c2d1a43a623b6 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 13 Nov 2013 00:15:59 -0800 Subject: [PATCH 265/331] Changing back to not break API --- src/main/scala/shark/SharkContext.scala | 48 +++++++++---------- src/main/scala/shark/SharkDriver.scala | 4 +- .../scala/shark/api/JavaSharkContext.scala | 2 +- src/main/scala/shark/api/TableRDD.scala | 2 +- .../scala/shark/api/TableRDDGenerated.scala | 46 +++++++++--------- .../scala/shark/execution/SparkTask.scala | 8 ++-- src/test/scala/shark/SQLSuite.scala | 8 ++-- 7 files changed, 59 insertions(+), 59 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 0143bd92..fe32d7d6 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -96,7 +96,7 @@ class SharkContext( * Execute a SQL command and return the results as a TableRDD. The SQL command must be * a SELECT statement. */ - def sqlRowRdd(cmd: String): RowRDD = { + def sql2rdd(cmd: String): TableRDD = { SparkEnv.set(sparkEnv) SessionState.start(sessionState) val driver = new SharkDriver(hiveconf) @@ -114,109 +114,109 @@ class SharkContext( * NB: These are auto-generated using resources/tablerdd/table_rdd_generators.py */ def sqlSeqRdd(cmd: String): RDD[Seq[Any]] = - new TableSeqRDD(sqlRowRdd(cmd)) + new TableSeqRDD(sql2rdd(cmd)) /** * Execute a SQL command and return the results as a RDD of Tuple. The SQL command must be * a SELECT statement. */ def sqlRdd[T](cmd: String): RDD[Tuple1[T]] = - new TableRDD1[T](sqlRowRdd(cmd)) + new TableRDD1[T](sql2rdd(cmd)) def sqlRdd[T1: M, T2: M](cmd: String): RDD[Tuple2[T1, T2]] = - new TableRDD2[T1, T2](sqlRowRdd(cmd), + new TableRDD2[T1, T2](sql2rdd(cmd), Seq(m[T1], m[T2])) def sqlRdd[T1: M, T2: M, T3: M](cmd: String): RDD[Tuple3[T1, T2, T3]] = - new TableRDD3[T1, T2, T3](sqlRowRdd(cmd), + new TableRDD3[T1, T2, T3](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3])) def sqlRdd[T1: M, T2: M, T3: M, T4: M](cmd: String): RDD[Tuple4[T1, T2, T3, T4]] = - new TableRDD4[T1, T2, T3, T4](sqlRowRdd(cmd), + new TableRDD4[T1, T2, T3, T4](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M](cmd: String): RDD[Tuple5[T1, T2, T3, T4, T5]] = - new TableRDD5[T1, T2, T3, T4, T5](sqlRowRdd(cmd), + new TableRDD5[T1, T2, T3, T4, T5](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](cmd: String): RDD[Tuple6[T1, T2, T3, T4, T5, T6]] = - new TableRDD6[T1, T2, T3, T4, T5, T6](sqlRowRdd(cmd), + new TableRDD6[T1, T2, T3, T4, T5, T6](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M](cmd: String): RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = - new TableRDD7[T1, T2, T3, T4, T5, T6, T7](sqlRowRdd(cmd), + new TableRDD7[T1, T2, T3, T4, T5, T6, T7](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M](cmd: String): RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = - new TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](sqlRowRdd(cmd), + new TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M](cmd: String): RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = - new TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](sqlRowRdd(cmd), + new TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M](cmd: String): RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = - new TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](sqlRowRdd(cmd), + new TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M](cmd: String): RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = - new TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](sqlRowRdd(cmd), + new TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M](cmd: String): RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = - new TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](sqlRowRdd(cmd), + new TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M](cmd: String): RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = - new TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](sqlRowRdd(cmd), + new TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M](cmd: String): RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = - new TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](sqlRowRdd(cmd), + new TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M](cmd: String): RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = - new TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](sqlRowRdd(cmd), + new TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M](cmd: String): RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = - new TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](sqlRowRdd(cmd), + new TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M](cmd: String): RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = - new TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](sqlRowRdd(cmd), + new TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17])) def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M](cmd: String): RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = - new TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](sqlRowRdd(cmd), + new TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) @@ -225,7 +225,7 @@ class SharkContext( RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19]] = new TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19](sqlRowRdd(cmd), + T19](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) @@ -234,7 +234,7 @@ class SharkContext( RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20]] = new TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20](sqlRowRdd(cmd), + T19, T20](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) @@ -243,7 +243,7 @@ class SharkContext( RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21]] = new TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21](sqlRowRdd(cmd), + T19, T20, T21](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) @@ -252,7 +252,7 @@ class SharkContext( RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22]] = new TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22](sqlRowRdd(cmd), + T19, T20, T21, T22](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 1812cde4..1869c946 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.{SerDe, SerDeUtils} import org.apache.hadoop.util.StringUtils -import shark.api.RowRDD +import shark.api.TableRDD import shark.api.QueryExecutionException import shark.execution.{SharkDDLTask, SharkDDLWork, SharkExplainTask, SharkExplainWork, SparkTask, SparkWork} @@ -143,7 +143,7 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe super.init() } - def tableRdd(cmd: String): Option[RowRDD] = { + def tableRdd(cmd: String): Option[TableRDD] = { useTableRddSink = true val response = run(cmd) // Throw an exception if there is an error in query processing. diff --git a/src/main/scala/shark/api/JavaSharkContext.scala b/src/main/scala/shark/api/JavaSharkContext.scala index 53d47447..9ec2914a 100644 --- a/src/main/scala/shark/api/JavaSharkContext.scala +++ b/src/main/scala/shark/api/JavaSharkContext.scala @@ -78,7 +78,7 @@ class JavaSharkContext(val sharkCtx: SharkContext) extends JavaSparkContext(shar * Execute the command and return the results as a TableRDD. */ def sql2rdd(cmd: String): JavaTableRDD = { - val rdd = sharkCtx.sqlRowRdd(cmd) + val rdd = sharkCtx.sql2rdd(cmd) new JavaTableRDD(rdd, rdd.schema) } diff --git a/src/main/scala/shark/api/TableRDD.scala b/src/main/scala/shark/api/TableRDD.scala index b09b484f..17aafaba 100644 --- a/src/main/scala/shark/api/TableRDD.scala +++ b/src/main/scala/shark/api/TableRDD.scala @@ -27,7 +27,7 @@ import shark.execution.serialization.KryoSerializer import org.apache.spark.{Partition, TaskContext} import org.apache.spark.rdd.RDD -class RowRDD( +class TableRDD( prev: RDD[Any], val schema: Array[ColumnDesc], @transient oi: ObjectInspector, diff --git a/src/main/scala/shark/api/TableRDDGenerated.scala b/src/main/scala/shark/api/TableRDDGenerated.scala index c6a433bb..fb37f6fe 100644 --- a/src/main/scala/shark/api/TableRDDGenerated.scala +++ b/src/main/scala/shark/api/TableRDDGenerated.scala @@ -20,7 +20,7 @@ package shark.api import org.apache.spark.rdd.RDD import org.apache.spark.{TaskContext, Partition} -class TableSeqRDD(prev: RowRDD) +class TableSeqRDD(prev: TableRDD) extends RDD[Seq[Any]](prev) { def getSchema = prev.schema @@ -34,7 +34,7 @@ class TableSeqRDD(prev: RowRDD) } } -class TableRDD1[T1](prev: RowRDD) +class TableRDD1[T1](prev: TableRDD) extends RDD[Tuple1[T1]](prev) { def getSchema = prev.schema @@ -53,7 +53,7 @@ class TableRDD1[T1](prev: RowRDD) } } -class TableRDD2[T1, T2](prev: RowRDD, +class TableRDD2[T1, T2](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple2[T1, T2]](prev) { def schema = prev.schema @@ -76,7 +76,7 @@ class TableRDD2[T1, T2](prev: RowRDD, } } -class TableRDD3[T1, T2, T3](prev: RowRDD, +class TableRDD3[T1, T2, T3](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple3[T1, T2, T3]](prev) { def schema = prev.schema @@ -100,7 +100,7 @@ class TableRDD3[T1, T2, T3](prev: RowRDD, } } -class TableRDD4[T1, T2, T3, T4](prev: RowRDD, +class TableRDD4[T1, T2, T3, T4](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple4[T1, T2, T3, T4]](prev) { def schema = prev.schema @@ -124,7 +124,7 @@ class TableRDD4[T1, T2, T3, T4](prev: RowRDD, } } -class TableRDD5[T1, T2, T3, T4, T5](prev: RowRDD, +class TableRDD5[T1, T2, T3, T4, T5](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple5[T1, T2, T3, T4, T5]](prev) { def schema = prev.schema @@ -148,7 +148,7 @@ class TableRDD5[T1, T2, T3, T4, T5](prev: RowRDD, } } -class TableRDD6[T1, T2, T3, T4, T5, T6](prev: RowRDD, +class TableRDD6[T1, T2, T3, T4, T5, T6](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple6[T1, T2, T3, T4, T5, T6]](prev) { def schema = prev.schema @@ -173,7 +173,7 @@ class TableRDD6[T1, T2, T3, T4, T5, T6](prev: RowRDD, } } -class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: RowRDD, +class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]](prev) { def schema = prev.schema @@ -198,7 +198,7 @@ class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: RowRDD, } } -class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: RowRDD, +class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]](prev) { def schema = prev.schema @@ -223,7 +223,7 @@ class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: RowRDD, } } -class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: RowRDD, +class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]](prev) { def schema = prev.schema @@ -249,7 +249,7 @@ class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: RowRDD, } } -class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: RowRDD, +class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]](prev) { def schema = prev.schema @@ -275,7 +275,7 @@ class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: RowRDD, } } -class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: RowRDD, +class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]](prev) { def schema = prev.schema @@ -301,7 +301,7 @@ class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: RowRDD, } } -class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: RowRDD, +class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]](prev) { def schema = prev.schema @@ -328,7 +328,7 @@ class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: RowRDD } } -class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: RowRDD, +class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]](prev) { def schema = prev.schema @@ -355,7 +355,7 @@ class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: R } } -class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: RowRDD, +class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]](prev) { def schema = prev.schema @@ -382,7 +382,7 @@ class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](pr } } -class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: RowRDD, +class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]](prev) { def schema = prev.schema @@ -410,7 +410,7 @@ class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: RowRDD, +class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]](prev) { def schema = prev.schema @@ -438,7 +438,7 @@ class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: RowRDD, +class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { def schema = prev.schema @@ -466,7 +466,7 @@ class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: RowRDD, +class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { def schema = prev.schema @@ -496,7 +496,7 @@ class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19](prev: RowRDD, +T19](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19]](prev) { @@ -529,7 +529,7 @@ T19](prev: RowRDD, } class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20](prev: RowRDD, +T19, T20](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20]](prev) { @@ -562,7 +562,7 @@ T19, T20](prev: RowRDD, } class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20, T21](prev: RowRDD, +T19, T20, T21](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21]](prev) { @@ -596,7 +596,7 @@ T19, T20, T21](prev: RowRDD, } class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20, T21, T22](prev: RowRDD, +T19, T20, T21, T22](prev: TableRDD, mans: Seq[ClassManifest[_]]) extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22]](prev) { diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index ca5dd853..04dea899 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.rdd.RDD -import shark.api.RowRDD +import shark.api.TableRDD import shark.{LogHelper, SharkEnv} @@ -49,9 +49,9 @@ extends java.io.Serializable private[shark] class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { - private var _tableRdd: Option[RowRDD] = None + private var _tableRdd: Option[TableRDD] = None - def tableRdd: Option[RowRDD] = _tableRdd + def tableRdd: Option[TableRDD] = _tableRdd override def execute(driverContext: DriverContext): Int = { logDebug("Executing " + this.getClass.getName) @@ -104,7 +104,7 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { } if (terminalOp.isInstanceOf[TableRddSinkOperator]) { - _tableRdd = Some(new RowRDD(sinkRdd, work.resultSchema, terminalOp.objectInspector, limit)) + _tableRdd = Some(new TableRDD(sinkRdd, work.resultSchema, terminalOp.objectInspector, limit)) } 0 diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index a8e5dcca..d3832478 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -834,9 +834,9 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("sql2rdd") { - var rdd = sc.sqlRowRdd("select * from test") + var rdd = sc.sql2rdd("select * from test") assert(rdd.count === 500) - rdd = sc.sqlRowRdd("select * from test_cached") + rdd = sc.sql2rdd("select * from test_cached") assert(rdd.count === 500) val collected = rdd.map(r => r.getInt(0)).collect().sortWith(_ < _) assert(collected(0) === 0) @@ -845,7 +845,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("null values in sql2rdd") { - val nullsRdd = sc.sqlRowRdd("select * from test_null where key is null") + val nullsRdd = sc.sql2rdd("select * from test_null where key is null") val nulls = nullsRdd.map(r => r.getInt(0)).collect() assert(nulls(0) === null) assert(nulls.size === 10) @@ -857,7 +857,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } test("sql2rdd exception") { - val e = intercept[QueryExecutionException] { sc.sqlRowRdd("asdfasdfasdfasdf") } + val e = intercept[QueryExecutionException] { sc.sql2rdd("asdfasdfasdfasdf") } e.getMessage.contains("semantic") } } From b1352fd87b2e6138f399be42fa16335af023152d Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 14 Nov 2013 01:21:49 -0800 Subject: [PATCH 266/331] implicit conversion from RDD to RDDTable --- src/main/scala/shark/api/RDDTable.scala | 82 +++++++++++++++++++++++++ 1 file changed, 82 insertions(+) diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index d93fac1d..8d251368 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -19,6 +19,88 @@ package shark.api import org.apache.spark.rdd.RDD +object RDDTableImplicits { + private type M[T] = ClassManifest[T] + + implicit def rddToTable2[T1: M, T2: M](rdd: RDD[(T1, T2)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable3[T1: M, T2: M, T3: M](rdd: RDD[(T1, T2, T3)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable4[T1: M, T2: M, T3: M, T4: M](rdd: RDD[(T1, T2, T3, T4)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable5[T1: M, T2: M, T3: M, T4: M, T5: M] + (rdd: RDD[(T1, T2, T3, T4, T5)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable6[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable7[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable8[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable9[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable10[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable11[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable12[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]): RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable13[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable14[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable15[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable16[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable17[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable18[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable19[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable20[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable21[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21)]): + RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable22[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, + T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22)]): + RDDTableFunctions = RDDTable(rdd) +} object RDDTable { From 4e522b1f9d6864791e4057127455edb7990a0090 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 20 Nov 2013 18:13:23 -0800 Subject: [PATCH 267/331] RDD[Seq] actually used inside RDDTableFunctions --- .../resources/tablerdd/rddtable_generator.py | 93 +++++++++++++++++++ 1 file changed, 93 insertions(+) create mode 100755 src/main/resources/tablerdd/rddtable_generator.py diff --git a/src/main/resources/tablerdd/rddtable_generator.py b/src/main/resources/tablerdd/rddtable_generator.py new file mode 100755 index 00000000..ab80b4b5 --- /dev/null +++ b/src/main/resources/tablerdd/rddtable_generator.py @@ -0,0 +1,93 @@ +#!/usr/bin/python +from string import Template +import sys + +## This script generates the 22 functions needed to create sqlToRdd's + +p = sys.stdout + +# e.g. createList(1,3, "T[", "]", ",") gives T[1],T[2],T[3] +def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, indent = 0): + res = "" + oneLine = res + for y in range(start,stop+1): + res += prefix + str(y) + suffix + oneLine += prefix + str(y) + suffix + if y != stop: + res += sep + oneLine += sep + if len(oneLine) > newlineAfter: + res += "\n" + " "*indent + oneLine = "" + return res + +### The SparkContext declaration + +prefix = """ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.api + +// *** This file is auto-generated from rddtable_generator.py *** + +import org.apache.spark.rdd.RDD + +object RDDTableImplicits { + private type M[T] = ClassManifest[T] + +""" + +p.write(prefix) + +for x in range(2,23): + + tableClass = Template( +""" + implicit def rddToTable2[$tmlist] + (rdd: RDD[($tlist)]): RDDTableFunctions = RDDTable(rdd) + +""").substitute(tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) + p.write(tableClass) + +prefix = """ +} + +object RDDTable { + + private type M[T] = ClassManifest[T] + private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) +""" + +p.write(prefix) + +for x in range(2,23): + + tableClass = Template( +""" + def apply[$tmlist] + (rdd: RDD[($tlist)]) = { + val rddSeq: RDD[Seq[_]] = rdd.map(_.productIterator.toList) + new RDDTableFunctions(rddSeq, Seq($mtlist)) + } + +""").substitute(tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4), + mtlist = createList(1, x, "m[T", "]", ", ", indent=4)) + p.write(tableClass) + + +p.write("}\n") From e796442ee1ac6475bc833f88609b30f61a84ed27 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 20 Nov 2013 18:13:29 -0800 Subject: [PATCH 268/331] RDD[Seq] actually used inside RDDTableFunctions --- .../resources/tablerdd/rddtable_generator.py | 7 +- src/main/scala/shark/api/RDDTable.scala | 399 +++++++++++------- .../scala/shark/api/RDDTableFunctions.scala | 4 +- 3 files changed, 250 insertions(+), 160 deletions(-) diff --git a/src/main/resources/tablerdd/rddtable_generator.py b/src/main/resources/tablerdd/rddtable_generator.py index ab80b4b5..7018a6e6 100755 --- a/src/main/resources/tablerdd/rddtable_generator.py +++ b/src/main/resources/tablerdd/rddtable_generator.py @@ -58,10 +58,10 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind tableClass = Template( """ - implicit def rddToTable2[$tmlist] + implicit def rddToTable$num[$tmlist] (rdd: RDD[($tlist)]): RDDTableFunctions = RDDTable(rdd) -""").substitute(tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) +""").substitute(num = x, tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) p.write(tableClass) prefix = """ @@ -81,7 +81,8 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind """ def apply[$tmlist] (rdd: RDD[($tlist)]) = { - val rddSeq: RDD[Seq[_]] = rdd.map(_.productIterator.toList) + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) new RDDTableFunctions(rddSeq, Seq($mtlist)) } diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 8d251368..e76ea0f8 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -17,89 +17,119 @@ package shark.api +// *** This file is auto-generated from rddtable_generator.py *** + import org.apache.spark.rdd.RDD object RDDTableImplicits { private type M[T] = ClassManifest[T] - implicit def rddToTable2[T1: M, T2: M](rdd: RDD[(T1, T2)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable3[T1: M, T2: M, T3: M](rdd: RDD[(T1, T2, T3)]): RDDTableFunctions = RDDTable(rdd) + implicit def rddToTable2[T1: M, T2: M] + (rdd: RDD[(T1, T2)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable3[T1: M, T2: M, T3: M] + (rdd: RDD[(T1, T2, T3)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable4[T1: M, T2: M, T3: M, T4: M] + (rdd: RDD[(T1, T2, T3, T4)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable4[T1: M, T2: M, T3: M, T4: M](rdd: RDD[(T1, T2, T3, T4)]): RDDTableFunctions = RDDTable(rdd) implicit def rddToTable5[T1: M, T2: M, T3: M, T4: M, T5: M] (rdd: RDD[(T1, T2, T3, T4, T5)]): RDDTableFunctions = RDDTable(rdd) + implicit def rddToTable6[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M] (rdd: RDD[(T1, T2, T3, T4, T5, T6)]): RDDTableFunctions = RDDTable(rdd) + implicit def rddToTable7[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]): RDDTableFunctions = RDDTable(rdd) + implicit def rddToTable8[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]): RDDTableFunctions = RDDTable(rdd) + implicit def rddToTable9[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]): RDDTableFunctions = RDDTable(rdd) + implicit def rddToTable10[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable11[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M] + + implicit def rddToTable11[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable12[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M] + + implicit def rddToTable12[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable13[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable14[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable15[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable16[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable17[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable18[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable19[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable20[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable21[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21)]): - RDDTableFunctions = RDDTable(rdd) - - implicit def rddToTable22[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22)]): - RDDTableFunctions = RDDTable(rdd) + + implicit def rddToTable13[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable14[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable15[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable16[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable17[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable18[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable19[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable20[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable21[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21)]): RDDTableFunctions = RDDTable(rdd) + + + implicit def rddToTable22[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M, T22: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22)]): RDDTableFunctions = RDDTable(rdd) + + } object RDDTable { @@ -107,148 +137,207 @@ object RDDTable { private type M[T] = ClassManifest[T] private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) - /* - * NB: These functions are auto-generated using resources/tablerdd/table_rdd_generators.py - */ + def apply[T1: M, T2: M] + (rdd: RDD[(T1, T2)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2])) + } - def apply[T1: M, T2: M](rdd: RDD[(T1, T2)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2])) + + def apply[T1: M, T2: M, T3: M] + (rdd: RDD[(T1, T2, T3)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3])) } - def apply[T1: M, T2: M, T3: M](rdd: RDD[(T1, T2, T3)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3])) + + def apply[T1: M, T2: M, T3: M, T4: M] + (rdd: RDD[(T1, T2, T3, T4)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4])) } - def apply[T1: M, T2: M, T3: M, T4: M](rdd: RDD[(T1, T2, T3, T4)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M] + (rdd: RDD[(T1, T2, T3, T4, T5)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M](rdd: RDD[(T1, T2, T3, T4, T5)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](rdd: RDD[(T1, T2, T3, T4, T5, T6)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M](rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13])) + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M](rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, - T18)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) - } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], + m[T20])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], + m[T20], m[T21])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M]( - rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22)]) = { - new RDDTableFunctions(rdd.asInstanceOf[RDD[Product]], - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) + + def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, + T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, + T20: M, T21: M, T22: M] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22)]) = { + val cm = implicitly[Manifest[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], + m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], + m[T20], m[T21], m[T22])) } + } diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 43c686a6..cf54b08f 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -28,7 +28,7 @@ import shark.memstore2.{CacheType, TablePartitionStats, TablePartition, TablePar import shark.util.HiveUtils -class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { +class RDDTableFunctions(self: RDD[Seq[_]], manifests: Seq[ClassManifest[_]]) { def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { require(fields.size == this.manifests.size, @@ -47,7 +47,7 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { for (p <- iter) { builder.incrementRowCount() // TODO: this is not the most efficient code to do the insertion ... - p.productIterator.zipWithIndex.foreach { case (v, i) => + p.zipWithIndex.foreach { case (v, i) => builder.append(i, v.asInstanceOf[Object], ois(i)) } } From 2b9b7ff9e3db6a30bf9043c413e6db0ba866b6eb Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 14 Nov 2013 00:35:53 -0800 Subject: [PATCH 269/331] TGF --- src/main/java/shark/tgf/Schema.java | 26 ++++ src/main/scala/shark/SharkContext.scala | 36 ++++- src/main/scala/shark/api/DataTypes.java | 1 + .../scala/shark/api/RDDTableFunctions.scala | 1 - src/main/scala/shark/tgf/TGF.scala | 133 ++++++++++++++++++ 5 files changed, 195 insertions(+), 2 deletions(-) create mode 100644 src/main/java/shark/tgf/Schema.java create mode 100644 src/main/scala/shark/tgf/TGF.scala diff --git a/src/main/java/shark/tgf/Schema.java b/src/main/java/shark/tgf/Schema.java new file mode 100644 index 00000000..7978be3e --- /dev/null +++ b/src/main/java/shark/tgf/Schema.java @@ -0,0 +1,26 @@ +/* + * Copyright (C) 2013 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.tgf; + +import java.lang.annotation.*; + +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.METHOD) +public @interface Schema { + String spec(); +} \ No newline at end of file diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index fe32d7d6..78d031cb 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -34,6 +34,7 @@ import org.apache.spark.{SparkContext, SparkEnv} import shark.api._ import org.apache.spark.rdd.RDD +import shark.tgf.TGF class SharkContext( @@ -108,6 +109,33 @@ class SharkContext( } } + def tableRdd(tableName: String): RDD[_] = { + val rdd = sql2rdd("SELECT * FROM " + tableName) + rdd.schema.size match { + case 2 => new TableRDD2(rdd, Seq()) + case 3 => new TableRDD3(rdd, Seq()) + case 4 => new TableRDD4(rdd, Seq()) + case 5 => new TableRDD5(rdd, Seq()) + case 6 => new TableRDD6(rdd, Seq()) + case 7 => new TableRDD7(rdd, Seq()) + case 8 => new TableRDD8(rdd, Seq()) + case 9 => new TableRDD9(rdd, Seq()) + case 10 => new TableRDD10(rdd, Seq()) + case 11 => new TableRDD11(rdd, Seq()) + case 12 => new TableRDD12(rdd, Seq()) + case 13 => new TableRDD13(rdd, Seq()) + case 14 => new TableRDD14(rdd, Seq()) + case 15 => new TableRDD15(rdd, Seq()) + case 16 => new TableRDD16(rdd, Seq()) + case 17 => new TableRDD17(rdd, Seq()) + case 18 => new TableRDD18(rdd, Seq()) + case 19 => new TableRDD19(rdd, Seq()) + case 20 => new TableRDD20(rdd, Seq()) + case 21 => new TableRDD21(rdd, Seq()) + case 22 => new TableRDD22(rdd, Seq()) + case _ => new TableSeqRDD(rdd) + } + } /** * Execute a SQL command and return the results as a RDD of Seq. The SQL command must be * a SELECT statement. This is useful if the table has more than 22 columns (more than fits in tuples) @@ -263,7 +291,13 @@ class SharkContext( * @param maxRows The max number of rows to retrieve for the result set. * @return A ResultSet object with both the schema and the query results. */ - def runSql(cmd: String, maxRows: Int = 1000): ResultSet = { + def runSql(cmd2: String, maxRows: Int = 1000): ResultSet = { + var cmd = cmd2 + if (cmd.trim.toLowerCase().startsWith("generate")) { + val (rdd, tableName, colnames) = TGF.parseInvokeTGF(cmd.trim, this) + cmd = "select * from " + tableName + " limit 0" + } + SparkEnv.set(sparkEnv) val cmd_trimmed: String = cmd.trim() diff --git a/src/main/scala/shark/api/DataTypes.java b/src/main/scala/shark/api/DataTypes.java index 1f71eb05..36871985 100644 --- a/src/main/scala/shark/api/DataTypes.java +++ b/src/main/scala/shark/api/DataTypes.java @@ -26,6 +26,7 @@ import scala.reflect.ClassManifest$; import org.apache.hadoop.hive.serde.Constants; +import scala.reflect.Manifest; /** * List of data types defined in Shark APIs. diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index cf54b08f..1109deab 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -27,7 +27,6 @@ import shark.{SharkContext, SharkEnv} import shark.memstore2.{CacheType, TablePartitionStats, TablePartition, TablePartitionBuilder} import shark.util.HiveUtils - class RDDTableFunctions(self: RDD[Seq[_]], manifests: Seq[ClassManifest[_]]) { def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala new file mode 100644 index 00000000..1d4d0bc7 --- /dev/null +++ b/src/main/scala/shark/tgf/TGF.scala @@ -0,0 +1,133 @@ +/* + * Copyright (C) 2013 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.tgf + +import java.sql.Timestamp +import java.util.Date +import scala.util.parsing.combinator._ +import org.apache.spark.rdd._ +import shark.SharkContext +import shark.api.RDDTableFunctions + +class TGFParser extends JavaTokenParsers { + + /* Code to enable case-insensitive modifiers to strings, e.g. "DataBricks".ci will match "databricks" */ + class MyString(str: String) { + def ci: Parser[String] = ("(?i)" + str).r + } + + implicit def stringToRichString(str: String): MyString = new MyString(str) + + def gentgf: Parser[Tuple3[String, String, List[String]]] = { + ((("GENERATE TABLE".ci ~> ident) <~ "USING".ci) ~ methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ + { case (id1 ~ id2) ~ x => (id1, id2, x.asInstanceOf[List[String]]) } + } + + def schema: Parser[List[Tuple2[String,String]]] = repsep(nameType, ",") + + def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } + + def param: Parser[Any] = stringLiteral | floatingPointNumber | decimalNumber | ident + + def methodName: Parser[String] = + """[a-zA-Z_][\w\.]*""".r + +} + +object KulTGF { + @Schema(spec = "name string, year int") + def apply(t1: RDD[(Int, String)], n: Int): RDD[(String, Int)] = { + t1.map{ case (i, s) => Tuple2(s, (i + n))} + } +} + +object TGF { + + val parser = new TGFParser + + // GENERATE TABLE minTable USING tgf(sales, dvd, 15, 3); + + def constructParams(tgfName: String, paramStrs: Seq[String], sc: SharkContext): + Tuple2[RDD[Product], Seq[Tuple2[String,String]]] = { + val tgfClazz = this.getClass.getClassLoader.loadClass(tgfName) + val methods = tgfClazz.getDeclaredMethods.filter(_.getName == "apply") + + if (methods.length < 1) throw new IllegalArgumentException("TGF " + tgfName + " needs to implement apply()") + + val method: java.lang.reflect.Method = methods(0) + + val typeNames: Seq[String] = method.getParameterTypes.toList.map(_.toString) + + val colSchema = parser.parseAll(parser.schema, method.getAnnotation(classOf[Schema]).spec()).get + + + if (colSchema.length != typeNames.length) + throw new IllegalArgumentException("Need schema annotation with " + typeNames.length + " columns") + + if (paramStrs.length != typeNames.length) throw new IllegalArgumentException("Expecting " + typeNames.length + + " parameters to " + tgfName + ", got " + paramStrs.length) + + val params = (paramStrs.toList zip typeNames.toList).map { + case (param: String, tpe: String) if (tpe.startsWith("class org.apache.spark.rdd.RDD")) => sc.tableRdd(param) + case (param: String, tpe: String) if (tpe.startsWith("long")) => param.toLong + case (param: String, tpe: String) if (tpe.startsWith("int")) => param.toInt + case (param: String, tpe: String) if (tpe.startsWith("double")) => param.toDouble + case (param: String, tpe: String) if (tpe.startsWith("float")) => param.toFloat + case (param: String, tpe: String) if (tpe.startsWith("class String")) => param + } + + println("### params " + params) + val tgfRes: RDD[Product] = method.invoke(null, params.asInstanceOf[List[Object]]:_*).asInstanceOf[RDD[Product]] + println("### created " + tgfRes) + + Tuple2(tgfRes, colSchema) + } + + def parseInvokeTGF(sql: String, sc: SharkContext): Tuple3[RDD[_], String, Seq[Tuple2[String,String]]] = { + val ast = parser.parseAll(parser.gentgf, sql).get + val tableName = ast._1 + val tgfName = ast._2 + val paramStrings = ast._3 + val (rdd, schema) = constructParams(tgfName, paramStrings, sc) + + println("### rdd " + rdd) + println("### schema " + schema) + + val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) + helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) + (rdd, tableName, schema) + } + + def toManifest(tpe: String): ClassManifest[_] = { + if (tpe == "boolean") classManifest[java.lang.Boolean] + else if (tpe == "tinyint") classManifest[java.lang.Byte] + else if (tpe == "smallint") classManifest[java.lang.Short] + else if (tpe == "int") classManifest[java.lang.Integer] + else if (tpe == "bigint") classManifest[java.lang.Long] + else if (tpe == "float") classManifest[java.lang.Float] + else if (tpe == "double") classManifest[java.lang.Double] + else if (tpe == "string") classManifest[java.lang.String] + else if (tpe == "timestamp") classManifest[Timestamp] + else if (tpe == "date") classManifest[Date] + else throw new IllegalArgumentException("Unknown column type specified in schema (" + tpe + ")") + } +// def main(args: Array[String]) { +// println(parseInvokeTGF("GEnerate table foo using Kul(\"hej\", bkaha, 10)")) +// sys.exit(0) +// } +} From ee192e7d98cc985121b32ee66a8777c73f043454 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 14 Nov 2013 18:45:32 -0800 Subject: [PATCH 270/331] TGF stuff --- src/main/java/shark/tgf/Schema.java | 11 +++- src/main/scala/shark/SharkContext.scala | 8 ++- src/main/scala/shark/api/DataTypes.java | 1 - .../scala/shark/api/RDDTableFunctions.scala | 1 - src/main/scala/shark/api/TableRDD.scala | 7 +-- .../scala/shark/api/TableRDDGenerated.scala | 3 +- src/main/scala/shark/tgf/TGF.scala | 55 ++++++++++--------- 7 files changed, 49 insertions(+), 37 deletions(-) diff --git a/src/main/java/shark/tgf/Schema.java b/src/main/java/shark/tgf/Schema.java index 7978be3e..c571a15f 100644 --- a/src/main/java/shark/tgf/Schema.java +++ b/src/main/java/shark/tgf/Schema.java @@ -17,10 +17,17 @@ package shark.tgf; -import java.lang.annotation.*; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.ElementType; +import java.lang.annotation.Target; + +/** + * Schema annotation for TGFs, example syntax: @Schema(spec = "name string, age int") + */ @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.METHOD) public @interface Schema { String spec(); -} \ No newline at end of file +} diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 78d031cb..2122aeaa 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -31,9 +31,9 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.{SparkContext, SparkEnv} +import org.apache.spark.rdd.RDD import shark.api._ -import org.apache.spark.rdd.RDD import shark.tgf.TGF @@ -109,6 +109,11 @@ class SharkContext( } } + /** + * Takes a table name and generates an RDD of tuples of the right size for it, if more than 22 columns it uses Seqs + * @param tableName + * @return RDD of either Tuple or Seq (if the table contains more than 22 columns) + */ def tableRdd(tableName: String): RDD[_] = { val rdd = sql2rdd("SELECT * FROM " + tableName) rdd.schema.size match { @@ -136,6 +141,7 @@ class SharkContext( case _ => new TableSeqRDD(rdd) } } + /** * Execute a SQL command and return the results as a RDD of Seq. The SQL command must be * a SELECT statement. This is useful if the table has more than 22 columns (more than fits in tuples) diff --git a/src/main/scala/shark/api/DataTypes.java b/src/main/scala/shark/api/DataTypes.java index 36871985..1f71eb05 100644 --- a/src/main/scala/shark/api/DataTypes.java +++ b/src/main/scala/shark/api/DataTypes.java @@ -26,7 +26,6 @@ import scala.reflect.ClassManifest$; import org.apache.hadoop.hive.serde.Constants; -import scala.reflect.Manifest; /** * List of data types defined in Shark APIs. diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 1109deab..cf98902c 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -20,7 +20,6 @@ package shark.api import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.ql.metadata.Hive - import org.apache.spark.rdd.RDD import shark.{SharkContext, SharkEnv} diff --git a/src/main/scala/shark/api/TableRDD.scala b/src/main/scala/shark/api/TableRDD.scala index 17aafaba..89122a18 100644 --- a/src/main/scala/shark/api/TableRDD.scala +++ b/src/main/scala/shark/api/TableRDD.scala @@ -20,13 +20,14 @@ package shark.api import java.util.{List => JList} import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector, StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} import shark.execution.serialization.KryoSerializer import org.apache.spark.{Partition, TaskContext} import org.apache.spark.rdd.RDD + class TableRDD( prev: RDD[Any], val schema: Array[ColumnDesc], @@ -73,8 +74,4 @@ class TableRDD( case _ => throw new Exception("Only basic StructObjectInspector is supposed.") } } - } - - - diff --git a/src/main/scala/shark/api/TableRDDGenerated.scala b/src/main/scala/shark/api/TableRDDGenerated.scala index fb37f6fe..eec98e94 100644 --- a/src/main/scala/shark/api/TableRDDGenerated.scala +++ b/src/main/scala/shark/api/TableRDDGenerated.scala @@ -20,6 +20,7 @@ package shark.api import org.apache.spark.rdd.RDD import org.apache.spark.{TaskContext, Partition} + class TableSeqRDD(prev: TableRDD) extends RDD[Seq[Any]](prev) { @@ -54,7 +55,7 @@ class TableRDD1[T1](prev: TableRDD) } class TableRDD2[T1, T2](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]] = Seq()) extends RDD[Tuple2[T1, T2]](prev) { def schema = prev.schema diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 1d4d0bc7..6014e5d1 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -19,10 +19,14 @@ package shark.tgf import java.sql.Timestamp import java.util.Date + import scala.util.parsing.combinator._ -import org.apache.spark.rdd._ + +import org.apache.spark.rdd.RDD + +import shark.api.{QueryExecutionException, RDDTableFunctions} import shark.SharkContext -import shark.api.RDDTableFunctions + class TGFParser extends JavaTokenParsers { @@ -34,7 +38,7 @@ class TGFParser extends JavaTokenParsers { implicit def stringToRichString(str: String): MyString = new MyString(str) def gentgf: Parser[Tuple3[String, String, List[String]]] = { - ((("GENERATE TABLE".ci ~> ident) <~ "USING".ci) ~ methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ + (((("GENERATE".ci ~ "TABLE".ci) ~> ident) <~ "USING".ci) ~ methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ { case (id1 ~ id2) ~ x => (id1, id2, x.asInstanceOf[List[String]]) } } @@ -42,13 +46,14 @@ class TGFParser extends JavaTokenParsers { def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } - def param: Parser[Any] = stringLiteral | floatingPointNumber | decimalNumber | ident + def param: Parser[Any] = stringLiteral | floatingPointNumber | decimalNumber | ident | + failure("Expected a string, number, or identifier as parameters in TGF") def methodName: Parser[String] = """[a-zA-Z_][\w\.]*""".r - } +// Example TGF, should be removed. object KulTGF { @Schema(spec = "name string, year int") def apply(t1: RDD[(Int, String)], n: Int): RDD[(String, Int)] = { @@ -60,26 +65,31 @@ object TGF { val parser = new TGFParser - // GENERATE TABLE minTable USING tgf(sales, dvd, 15, 3); - - def constructParams(tgfName: String, paramStrs: Seq[String], sc: SharkContext): + def runTGF(tgfName: String, paramStrs: Seq[String], sc: SharkContext): Tuple2[RDD[Product], Seq[Tuple2[String,String]]] = { - val tgfClazz = this.getClass.getClassLoader.loadClass(tgfName) + val tgfClazz = try { + this.getClass.getClassLoader.loadClass(tgfName) + } catch { + case ex: ClassNotFoundException => throw new QueryExecutionException("Couldn't find TGF class: " + tgfName) + } + val methods = tgfClazz.getDeclaredMethods.filter(_.getName == "apply") - if (methods.length < 1) throw new IllegalArgumentException("TGF " + tgfName + " needs to implement apply()") + if (methods.length < 1) throw new QueryExecutionException("TGF " + tgfName + " needs to implement apply()") val method: java.lang.reflect.Method = methods(0) val typeNames: Seq[String] = method.getParameterTypes.toList.map(_.toString) - val colSchema = parser.parseAll(parser.schema, method.getAnnotation(classOf[Schema]).spec()).get - + val annotations = method.getAnnotation(classOf[Schema]).spec() + val colSchema = parser.parseAll(parser.schema, annotations).getOrElse( + throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)")) if (colSchema.length != typeNames.length) - throw new IllegalArgumentException("Need schema annotation with " + typeNames.length + " columns") + throw new QueryExecutionException("Need schema annotation with " + typeNames.length + " columns") - if (paramStrs.length != typeNames.length) throw new IllegalArgumentException("Expecting " + typeNames.length + + + if (paramStrs.length != typeNames.length) throw new QueryExecutionException("Expecting " + typeNames.length + " parameters to " + tgfName + ", got " + paramStrs.length) val params = (paramStrs.toList zip typeNames.toList).map { @@ -89,24 +99,21 @@ object TGF { case (param: String, tpe: String) if (tpe.startsWith("double")) => param.toDouble case (param: String, tpe: String) if (tpe.startsWith("float")) => param.toFloat case (param: String, tpe: String) if (tpe.startsWith("class String")) => param + case (param: String, tpe: String) => throw + new QueryExecutionException("Expected TGF parameter type: " + tpe + " (" + param + ")") } - println("### params " + params) val tgfRes: RDD[Product] = method.invoke(null, params.asInstanceOf[List[Object]]:_*).asInstanceOf[RDD[Product]] - println("### created " + tgfRes) Tuple2(tgfRes, colSchema) } def parseInvokeTGF(sql: String, sc: SharkContext): Tuple3[RDD[_], String, Seq[Tuple2[String,String]]] = { - val ast = parser.parseAll(parser.gentgf, sql).get + val ast = parser.parseAll(parser.gentgf, sql).getOrElse{throw new QueryExecutionException("TGF parse error: "+ sql)} val tableName = ast._1 val tgfName = ast._2 val paramStrings = ast._3 - val (rdd, schema) = constructParams(tgfName, paramStrings, sc) - - println("### rdd " + rdd) - println("### schema " + schema) + val (rdd, schema) = runTGF(tgfName, paramStrings, sc) val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) @@ -124,10 +131,6 @@ object TGF { else if (tpe == "string") classManifest[java.lang.String] else if (tpe == "timestamp") classManifest[Timestamp] else if (tpe == "date") classManifest[Date] - else throw new IllegalArgumentException("Unknown column type specified in schema (" + tpe + ")") + else throw new QueryExecutionException("Unknown column type specified in schema (" + tpe + ")") } -// def main(args: Array[String]) { -// println(parseInvokeTGF("GEnerate table foo using Kul(\"hej\", bkaha, 10)")) -// sys.exit(0) -// } } From 71f9b6ebe9ba468b8efe84c90f6225fd6891cd63 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sat, 16 Nov 2013 21:56:17 -0800 Subject: [PATCH 271/331] now separate TGF commands for saving and viewing the table --- src/main/scala/shark/SharkContext.scala | 3 +- src/main/scala/shark/tgf/TGF.scala | 94 +++++++++++++++++-------- 2 files changed, 65 insertions(+), 32 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 2122aeaa..a5977189 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -300,8 +300,7 @@ class SharkContext( def runSql(cmd2: String, maxRows: Int = 1000): ResultSet = { var cmd = cmd2 if (cmd.trim.toLowerCase().startsWith("generate")) { - val (rdd, tableName, colnames) = TGF.parseInvokeTGF(cmd.trim, this) - cmd = "select * from " + tableName + " limit 0" + return TGF.execute(cmd.trim, this) } SparkEnv.set(sparkEnv) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 6014e5d1..83825a96 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -18,17 +18,20 @@ package shark.tgf import java.sql.Timestamp -import java.util.Date +import java.util.{Random, Date} import scala.util.parsing.combinator._ import org.apache.spark.rdd.RDD -import shark.api.{QueryExecutionException, RDDTableFunctions} +import shark.api._ import shark.SharkContext +import scala.Tuple3 +import scala.Some +import scala.Tuple2 -class TGFParser extends JavaTokenParsers { +private class TGFParser extends JavaTokenParsers { /* Code to enable case-insensitive modifiers to strings, e.g. "DataBricks".ci will match "databricks" */ class MyString(str: String) { @@ -37,11 +40,24 @@ class TGFParser extends JavaTokenParsers { implicit def stringToRichString(str: String): MyString = new MyString(str) - def gentgf: Parser[Tuple3[String, String, List[String]]] = { - (((("GENERATE".ci ~ "TABLE".ci) ~> ident) <~ "USING".ci) ~ methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ - { case (id1 ~ id2) ~ x => (id1, id2, x.asInstanceOf[List[String]]) } + def tgf: Parser[Any] = saveTgf | basicTgf + + /** + * @return Tuple2 containing a TGF method name and a List of parameters as strings + */ + def basicTgf: Parser[Tuple2[String, List[String]]] = { + (("GENERATE".ci ~ "USING".ci) ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ + { case id1 ~ x => (id1, x.asInstanceOf[List[String]]) } } + /** + * @return Tuple3 containing a table name, TGF method name and a List of parameters as strings + */ + def saveTgf: Parser[Tuple3[String, String, List[String]]] = { + ((("GENERATE".ci ~ "USING".ci) ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("SAVE".ci ~ "AS".ci) ~> + ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } + } + def schema: Parser[List[Tuple2[String,String]]] = repsep(nameType, ",") def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } @@ -53,19 +69,11 @@ class TGFParser extends JavaTokenParsers { """[a-zA-Z_][\w\.]*""".r } -// Example TGF, should be removed. -object KulTGF { - @Schema(spec = "name string, year int") - def apply(t1: RDD[(Int, String)], n: Int): RDD[(String, Int)] = { - t1.map{ case (i, s) => Tuple2(s, (i + n))} - } -} - object TGF { - val parser = new TGFParser + private val parser = new TGFParser - def runTGF(tgfName: String, paramStrs: Seq[String], sc: SharkContext): + private def reflectInvoke(tgfName: String, paramStrs: Seq[String], sc: SharkContext): Tuple2[RDD[Product], Seq[Tuple2[String,String]]] = { val tgfClazz = try { this.getClass.getClassLoader.loadClass(tgfName) @@ -85,10 +93,6 @@ object TGF { val colSchema = parser.parseAll(parser.schema, annotations).getOrElse( throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)")) - if (colSchema.length != typeNames.length) - throw new QueryExecutionException("Need schema annotation with " + typeNames.length + " columns") - - if (paramStrs.length != typeNames.length) throw new QueryExecutionException("Expecting " + typeNames.length + " parameters to " + tgfName + ", got " + paramStrs.length) @@ -108,19 +112,33 @@ object TGF { Tuple2(tgfRes, colSchema) } - def parseInvokeTGF(sql: String, sc: SharkContext): Tuple3[RDD[_], String, Seq[Tuple2[String,String]]] = { - val ast = parser.parseAll(parser.gentgf, sql).getOrElse{throw new QueryExecutionException("TGF parse error: "+ sql)} - val tableName = ast._1 - val tgfName = ast._2 - val paramStrings = ast._3 - val (rdd, schema) = runTGF(tgfName, paramStrings, sc) + def execute(sql: String, sc: SharkContext): ResultSet = { + val ast = parser.parseAll(parser.tgf, sql).getOrElse{throw new QueryExecutionException("TGF parse error: "+ sql)} + + val (tableNameOpt, tgfName, params) = ast match { + case Tuple2(tgfName, params) => (None, tgfName.asInstanceOf[String], params.asInstanceOf[List[String]]) + case Tuple3(tableName, tgfName, params) => (Some(tableName.asInstanceOf[String]), tgfName.asInstanceOf[String], + params.asInstanceOf[List[String]]) + } + + val (resultRdd, schema) = reflectInvoke(tgfName, params, sc) - val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) - helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) - (rdd, tableName, schema) + val (sharkSchema, resultArr) = tableNameOpt match { + + case Some(tableName) => // materialize results + val helper = new RDDTableFunctions(resultRdd, schema.map{ case (_, tpe) => toManifest(tpe)}) + helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) + (Array[ColumnDesc](), Array[Array[Object]]()) + + case None => // return results + val newSchema = schema.map{ case (name, tpe) => new ColumnDesc(name, DataTypes.fromManifest(toManifest(tpe)))} + val res = resultRdd.collect().map{p => p.productIterator.map( _.asInstanceOf[Object] ).toArray} + (newSchema.toArray, res) + } + new ResultSet(sharkSchema, resultArr) } - def toManifest(tpe: String): ClassManifest[_] = { + private def toManifest(tpe: String): ClassManifest[_] = { if (tpe == "boolean") classManifest[java.lang.Boolean] else if (tpe == "tinyint") classManifest[java.lang.Byte] else if (tpe == "smallint") classManifest[java.lang.Short] @@ -134,3 +152,19 @@ object TGF { else throw new QueryExecutionException("Unknown column type specified in schema (" + tpe + ")") } } + + +// Example TGF, should be removed. +object Clustering { + import org.apache.spark.mllib.clustering._ + + @Schema(spec = "year int, state string, product string, sales double, cluster int") + def apply(sales: RDD[(Int, String, String, Double)], k: Int): + RDD[(Int, String, String, Double, Int)] = { + + val dataset = sales.map{ case (year, state, product, sales) => Array[Double](sales) } + val model = KMeans.train(dataset, k, 2, 2) + + sales.map{ case (year, state, product, sales) => (year, state, product, sales, model.predict(Array(sales))) } + } +} \ No newline at end of file From 7ff3eb57b68ae525550f43b319b98a39e90fb234 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sun, 17 Nov 2013 17:49:47 -0800 Subject: [PATCH 272/331] improved the syntax of TGFs --- src/main/scala/shark/tgf/TGF.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 83825a96..a31e6624 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -46,7 +46,7 @@ private class TGFParser extends JavaTokenParsers { * @return Tuple2 containing a TGF method name and a List of parameters as strings */ def basicTgf: Parser[Tuple2[String, List[String]]] = { - (("GENERATE".ci ~ "USING".ci) ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ + ("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ { case id1 ~ x => (id1, x.asInstanceOf[List[String]]) } } @@ -54,7 +54,7 @@ private class TGFParser extends JavaTokenParsers { * @return Tuple3 containing a table name, TGF method name and a List of parameters as strings */ def saveTgf: Parser[Tuple3[String, String, List[String]]] = { - ((("GENERATE".ci ~ "USING".ci) ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("SAVE".ci ~ "AS".ci) ~> + (("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("SAVE".ci ~ "AS".ci) ~> ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } } From fccc38fd3684c7b1c5a12619759f83231f0389f6 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 20 Nov 2013 10:34:38 -0800 Subject: [PATCH 273/331] TGFs semi-support dynamic schemas --- src/main/scala/shark/tgf/TGF.scala | 89 +++++++++++++++++++----------- 1 file changed, 56 insertions(+), 33 deletions(-) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index a31e6624..2f4835b9 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -30,6 +30,7 @@ import scala.Tuple3 import scala.Some import scala.Tuple2 +case class RDDSchema(rdd: RDD[Seq[_]], schema: Seq[Tuple2[String,String]]) private class TGFParser extends JavaTokenParsers { @@ -58,7 +59,7 @@ private class TGFParser extends JavaTokenParsers { ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } } - def schema: Parser[List[Tuple2[String,String]]] = repsep(nameType, ",") + def schema: Parser[Seq[Tuple2[String,String]]] = repsep(nameType, ",") def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } @@ -73,25 +74,45 @@ object TGF { private val parser = new TGFParser - private def reflectInvoke(tgfName: String, paramStrs: Seq[String], sc: SharkContext): - Tuple2[RDD[Product], Seq[Tuple2[String,String]]] = { + private def getMethod(tgfName: String, methodName: String) = { val tgfClazz = try { - this.getClass.getClassLoader.loadClass(tgfName) + Thread.currentThread().getContextClassLoader.loadClass(tgfName) } catch { case ex: ClassNotFoundException => throw new QueryExecutionException("Couldn't find TGF class: " + tgfName) } - val methods = tgfClazz.getDeclaredMethods.filter(_.getName == "apply") + val methods = tgfClazz.getDeclaredMethods.filter(_.getName == methodName) + if (methods.isEmpty) None else Some(methods(0)) + } - if (methods.length < 1) throw new QueryExecutionException("TGF " + tgfName + " needs to implement apply()") +// private def isOfType(obj: AnyRef, typeString: String) = { +// obj.getClass.getTy +// } + + private def getSchema(tgfOutput: Object, tgfName: String): Tuple2[RDD[Seq[_]], Seq[Tuple2[String,String]]] = { + if (tgfOutput.isInstanceOf[RDDSchema]) { + val rddSchema = tgfOutput.asInstanceOf[RDDSchema] + (rddSchema.rdd, rddSchema.schema) + } else if (tgfOutput.isInstanceOf[RDD[Product]]) { + val applyMethod = getMethod(tgfName, "apply") + if (applyMethod == None) throw new QueryExecutionException("TGF lacking apply() method") + + val annotations = applyMethod.get.getAnnotation(classOf[Schema]).spec() + if (annotations == None) throw new QueryExecutionException("No schema annotation found for TGF") + + val schema = parser.parseAll(parser.schema, annotations) + if (schema == None) throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)") + (tgfOutput.asInstanceOf[RDD[Product]].map(_.productIterator.toList), schema.get) + } else throw new QueryExecutionException("TGF output needs to be of type RDD or RDDSchema") + } - val method: java.lang.reflect.Method = methods(0) + private def reflectInvoke(tgfName: String, paramStrs: Seq[String], sc: SharkContext) = { - val typeNames: Seq[String] = method.getParameterTypes.toList.map(_.toString) + val applyMethodOpt = getMethod(tgfName, "apply") + if (applyMethodOpt.isEmpty) throw new QueryExecutionException("TGF " + tgfName + " needs to implement apply()") + val applyMethod = applyMethodOpt.get - val annotations = method.getAnnotation(classOf[Schema]).spec() - val colSchema = parser.parseAll(parser.schema, annotations).getOrElse( - throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)")) + val typeNames: Seq[String] = applyMethod.getParameterTypes.toList.map(_.toString) if (paramStrs.length != typeNames.length) throw new QueryExecutionException("Expecting " + typeNames.length + " parameters to " + tgfName + ", got " + paramStrs.length) @@ -107,9 +128,7 @@ object TGF { new QueryExecutionException("Expected TGF parameter type: " + tpe + " (" + param + ")") } - val tgfRes: RDD[Product] = method.invoke(null, params.asInstanceOf[List[Object]]:_*).asInstanceOf[RDD[Product]] - - Tuple2(tgfRes, colSchema) + applyMethod.invoke(null, params.asInstanceOf[List[Object]]:_*) } def execute(sql: String, sc: SharkContext): ResultSet = { @@ -121,18 +140,18 @@ object TGF { params.asInstanceOf[List[String]]) } - val (resultRdd, schema) = reflectInvoke(tgfName, params, sc) + val obj = reflectInvoke(tgfName, params, sc) + val (rdd, schema) = getSchema(obj, tgfName) val (sharkSchema, resultArr) = tableNameOpt match { - case Some(tableName) => // materialize results - val helper = new RDDTableFunctions(resultRdd, schema.map{ case (_, tpe) => toManifest(tpe)}) + val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) (Array[ColumnDesc](), Array[Array[Object]]()) case None => // return results val newSchema = schema.map{ case (name, tpe) => new ColumnDesc(name, DataTypes.fromManifest(toManifest(tpe)))} - val res = resultRdd.collect().map{p => p.productIterator.map( _.asInstanceOf[Object] ).toArray} + val res = rdd.collect().map{p => p.map( _.asInstanceOf[Object] ).toArray} (newSchema.toArray, res) } new ResultSet(sharkSchema, resultArr) @@ -153,18 +172,22 @@ object TGF { } } - -// Example TGF, should be removed. -object Clustering { - import org.apache.spark.mllib.clustering._ - - @Schema(spec = "year int, state string, product string, sales double, cluster int") - def apply(sales: RDD[(Int, String, String, Double)], k: Int): - RDD[(Int, String, String, Double, Int)] = { - - val dataset = sales.map{ case (year, state, product, sales) => Array[Double](sales) } - val model = KMeans.train(dataset, k, 2, 2) - - sales.map{ case (year, state, product, sales) => (year, state, product, sales, model.predict(Array(sales))) } - } -} \ No newline at end of file +//object NameOfTGF { +// import org.apache.spark.mllib.clustering._ +// +// // TGFs need to implement an apply() method. +// // The TGF has to have an apply function that takes any arbitrary primitive types and any number of RDDs. +// // When a TGF is invoked from Shark, every type of RDD is produced by converting Hive tables to RDDs +// // TGFs need to have a return type that is either an RDD[Product] or RDDSchema +// // The former case requires that the apply method has an annotation of the schema (see below) +// // In the latter case the schema is embedded in the RDDSchema function +// @Schema(spec = "year int, state string, product string, sales double, cluster int") +// def apply(sales: RDD[(Int, String, String, Double)], k: Int): +// RDD[(Int, String, String, Double, Int)] = { +// +// val dataset = sales.map{ case (year, state, product, sales) => Array[Double](sales) } +// val model = KMeans.train(dataset, k, 2, 2) +// +// sales.map{ case (year, state, product, sales) => (year, state, product, sales, model.predict(Array(sales))) } +// } +//} From 7812aee6533e06775feb92bf37c1cc9f0be5e918 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 20 Nov 2013 18:47:57 -0800 Subject: [PATCH 274/331] RDDSchema return type of RDD[Seq[_]] --- src/main/scala/shark/tgf/TGF.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 2f4835b9..564f0ed8 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -190,4 +190,18 @@ object TGF { // // sales.map{ case (year, state, product, sales) => (year, state, product, sales, model.predict(Array(sales))) } // } +// // Alternatively, using RDDSchema return time you can dynamically decide the columns and their types +// def apply(sales: RDD[(Int, String, String, Double)], k: Int): +// RDDSchema = { +// +// val dataset = sales.map{ case (year, state, product, sales) => Array[Double](sales) } +// val model = KMeans.train(dataset, k, 2, 2) +// +// val rdd = sales.map{ +// case (year, state, product, sales) => List(year, state, product, sales, model.predict(Array(sales))) +// } +// +// RDDSchema(rdd.asInstanceOf[RDD[Seq[_]]], +// List(("year","int"), ("state", "string"), ("product", "string"), ("sales", "double"), ("cluster", "int"))) +// } //} From 72e220c36a9b112430bfdccbe2dce0240cb4dba7 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 21 Nov 2013 15:43:42 -0800 Subject: [PATCH 275/331] Merging with alig/master --- src/main/scala/shark/SharkContext.scala | 113 ++++++++++-------------- 1 file changed, 47 insertions(+), 66 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index a5977189..fe62122b 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -31,10 +31,9 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.{SparkContext, SparkEnv} -import org.apache.spark.rdd.RDD import shark.api._ -import shark.tgf.TGF +import org.apache.spark.rdd.RDD class SharkContext( @@ -109,186 +108,173 @@ class SharkContext( } } - /** - * Takes a table name and generates an RDD of tuples of the right size for it, if more than 22 columns it uses Seqs - * @param tableName - * @return RDD of either Tuple or Seq (if the table contains more than 22 columns) - */ - def tableRdd(tableName: String): RDD[_] = { - val rdd = sql2rdd("SELECT * FROM " + tableName) - rdd.schema.size match { - case 2 => new TableRDD2(rdd, Seq()) - case 3 => new TableRDD3(rdd, Seq()) - case 4 => new TableRDD4(rdd, Seq()) - case 5 => new TableRDD5(rdd, Seq()) - case 6 => new TableRDD6(rdd, Seq()) - case 7 => new TableRDD7(rdd, Seq()) - case 8 => new TableRDD8(rdd, Seq()) - case 9 => new TableRDD9(rdd, Seq()) - case 10 => new TableRDD10(rdd, Seq()) - case 11 => new TableRDD11(rdd, Seq()) - case 12 => new TableRDD12(rdd, Seq()) - case 13 => new TableRDD13(rdd, Seq()) - case 14 => new TableRDD14(rdd, Seq()) - case 15 => new TableRDD15(rdd, Seq()) - case 16 => new TableRDD16(rdd, Seq()) - case 17 => new TableRDD17(rdd, Seq()) - case 18 => new TableRDD18(rdd, Seq()) - case 19 => new TableRDD19(rdd, Seq()) - case 20 => new TableRDD20(rdd, Seq()) - case 21 => new TableRDD21(rdd, Seq()) - case 22 => new TableRDD22(rdd, Seq()) - case _ => new TableSeqRDD(rdd) - } - } - /** * Execute a SQL command and return the results as a RDD of Seq. The SQL command must be * a SELECT statement. This is useful if the table has more than 22 columns (more than fits in tuples) - * NB: These are auto-generated using resources/tablerdd/table_rdd_generators.py + * NB: These are auto-generated using resources/tablerdd/SharkContext_sqlRdd_generator.py */ - def sqlSeqRdd(cmd: String): RDD[Seq[Any]] = + def sqlSeqRdd(cmd: String): RDD[Seq[Any]] = { new TableSeqRDD(sql2rdd(cmd)) + } /** * Execute a SQL command and return the results as a RDD of Tuple. The SQL command must be * a SELECT statement. */ - def sqlRdd[T](cmd: String): RDD[Tuple1[T]] = - new TableRDD1[T](sql2rdd(cmd)) def sqlRdd[T1: M, T2: M](cmd: String): - RDD[Tuple2[T1, T2]] = + RDD[Tuple2[T1, T2]] = { new TableRDD2[T1, T2](sql2rdd(cmd), Seq(m[T1], m[T2])) + } def sqlRdd[T1: M, T2: M, T3: M](cmd: String): - RDD[Tuple3[T1, T2, T3]] = + RDD[Tuple3[T1, T2, T3]] = { new TableRDD3[T1, T2, T3](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M](cmd: String): - RDD[Tuple4[T1, T2, T3, T4]] = + RDD[Tuple4[T1, T2, T3, T4]] = { new TableRDD4[T1, T2, T3, T4](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M](cmd: String): - RDD[Tuple5[T1, T2, T3, T4, T5]] = + RDD[Tuple5[T1, T2, T3, T4, T5]] = { new TableRDD5[T1, T2, T3, T4, T5](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](cmd: String): - RDD[Tuple6[T1, T2, T3, T4, T5, T6]] = + RDD[Tuple6[T1, T2, T3, T4, T5, T6]] = { new TableRDD6[T1, T2, T3, T4, T5, T6](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M](cmd: String): - RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = + RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { new TableRDD7[T1, T2, T3, T4, T5, T6, T7](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M](cmd: String): - RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = + RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { new TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M](cmd: String): - RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = + RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { new TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M](cmd: String): - RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = + RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { new TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M](cmd: String): - RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = + RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { new TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M](cmd: String): - RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = + RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { new TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M](cmd: String): - RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = + RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { new TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M](cmd: String): - RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = + RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { new TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M](cmd: String): - RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = + RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { new TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M](cmd: String): - RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = + RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { new TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M](cmd: String): - RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = + RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { new TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M](cmd: String): - RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = + RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { new TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M](cmd: String): RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19]] = + T19]] = { new TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M](cmd: String): RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20]] = + T19, T20]] = { new TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M](cmd: String): RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21]] = + T19, T20, T21]] = { new TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) + } def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M](cmd: String): RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22]] = + T19, T20, T21, T22]] = { new TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22](sql2rdd(cmd), Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) + } /** * Execute a SQL command and collect the results locally. @@ -297,12 +283,7 @@ class SharkContext( * @param maxRows The max number of rows to retrieve for the result set. * @return A ResultSet object with both the schema and the query results. */ - def runSql(cmd2: String, maxRows: Int = 1000): ResultSet = { - var cmd = cmd2 - if (cmd.trim.toLowerCase().startsWith("generate")) { - return TGF.execute(cmd.trim, this) - } - + def runSql(cmd: String, maxRows: Int = 1000): ResultSet = { SparkEnv.set(sparkEnv) val cmd_trimmed: String = cmd.trim() From c9df779b0155f30bdd7b7defe7789f6b1ba15a40 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 21 Nov 2013 15:48:05 -0800 Subject: [PATCH 276/331] Merging with alig/master --- src/main/scala/shark/SharkContext.scala | 40 ++++++++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index fe62122b..809dcde2 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -31,9 +31,10 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.{SparkContext, SparkEnv} +import org.apache.spark.rdd.RDD import shark.api._ -import org.apache.spark.rdd.RDD +import shark.tgf.TGF class SharkContext( @@ -92,6 +93,39 @@ class SharkContext( } } + /** + * Takes a table name and generates an RDD of tuples of the right size for it, if more than 22 columns it uses Seqs + * @param tableName + * @return RDD of either Tuple or Seq (if the table contains more than 22 columns) + */ + def tableRdd(tableName: String): RDD[_] = { + val rdd = sql2rdd("SELECT * FROM " + tableName) + rdd.schema.size match { + case 2 => new TableRDD2(rdd, Seq()) + case 3 => new TableRDD3(rdd, Seq()) + case 4 => new TableRDD4(rdd, Seq()) + case 5 => new TableRDD5(rdd, Seq()) + case 6 => new TableRDD6(rdd, Seq()) + case 7 => new TableRDD7(rdd, Seq()) + case 8 => new TableRDD8(rdd, Seq()) + case 9 => new TableRDD9(rdd, Seq()) + case 10 => new TableRDD10(rdd, Seq()) + case 11 => new TableRDD11(rdd, Seq()) + case 12 => new TableRDD12(rdd, Seq()) + case 13 => new TableRDD13(rdd, Seq()) + case 14 => new TableRDD14(rdd, Seq()) + case 15 => new TableRDD15(rdd, Seq()) + case 16 => new TableRDD16(rdd, Seq()) + case 17 => new TableRDD17(rdd, Seq()) + case 18 => new TableRDD18(rdd, Seq()) + case 19 => new TableRDD19(rdd, Seq()) + case 20 => new TableRDD20(rdd, Seq()) + case 21 => new TableRDD21(rdd, Seq()) + case 22 => new TableRDD22(rdd, Seq()) + case _ => new TableSeqRDD(rdd) + } + } + /** * Execute a SQL command and return the results as a TableRDD. The SQL command must be * a SELECT statement. @@ -284,6 +318,10 @@ class SharkContext( * @return A ResultSet object with both the schema and the query results. */ def runSql(cmd: String, maxRows: Int = 1000): ResultSet = { + if (cmd.trim.toLowerCase().startsWith("generate")) { + return TGF.execute(cmd.trim, this) + } + SparkEnv.set(sparkEnv) val cmd_trimmed: String = cmd.trim() From 69737d7aefa970075e47b4f3b7b904bcdc0b1bd5 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 21 Nov 2013 16:16:55 -0800 Subject: [PATCH 277/331] Reynold's PR comments addressed --- src/main/scala/shark/tgf/TGF.scala | 57 ++++++++++++++++++------------ 1 file changed, 35 insertions(+), 22 deletions(-) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 564f0ed8..f67dc0ce 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -18,7 +18,7 @@ package shark.tgf import java.sql.Timestamp -import java.util.{Random, Date} +import java.util.Date import scala.util.parsing.combinator._ @@ -26,9 +26,6 @@ import org.apache.spark.rdd.RDD import shark.api._ import shark.SharkContext -import scala.Tuple3 -import scala.Some -import scala.Tuple2 case class RDDSchema(rdd: RDD[Seq[_]], schema: Seq[Tuple2[String,String]]) @@ -58,7 +55,7 @@ private class TGFParser extends JavaTokenParsers { (("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("SAVE".ci ~ "AS".ci) ~> ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } } - + def schema: Parser[Seq[Tuple2[String,String]]] = repsep(nameType, ",") def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } @@ -85,9 +82,9 @@ object TGF { if (methods.isEmpty) None else Some(methods(0)) } -// private def isOfType(obj: AnyRef, typeString: String) = { -// obj.getClass.getTy -// } + // private def isOfType(obj: AnyRef, typeString: String) = { + // obj.getClass.getTy + // } private def getSchema(tgfOutput: Object, tgfName: String): Tuple2[RDD[Seq[_]], Seq[Tuple2[String,String]]] = { if (tgfOutput.isInstanceOf[RDDSchema]) { @@ -95,27 +92,41 @@ object TGF { (rddSchema.rdd, rddSchema.schema) } else if (tgfOutput.isInstanceOf[RDD[Product]]) { val applyMethod = getMethod(tgfName, "apply") - if (applyMethod == None) throw new QueryExecutionException("TGF lacking apply() method") + if (applyMethod == None) { + throw new QueryExecutionException("TGF lacking apply() method") + } val annotations = applyMethod.get.getAnnotation(classOf[Schema]).spec() - if (annotations == None) throw new QueryExecutionException("No schema annotation found for TGF") + if (annotations == None) { + throw new QueryExecutionException("No schema annotation found for TGF") + } val schema = parser.parseAll(parser.schema, annotations) - if (schema == None) throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)") + if (schema == None) { + throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)") + } + (tgfOutput.asInstanceOf[RDD[Product]].map(_.productIterator.toList), schema.get) - } else throw new QueryExecutionException("TGF output needs to be of type RDD or RDDSchema") + } else { + throw new QueryExecutionException("TGF output needs to be of type RDD or RDDSchema") + } } private def reflectInvoke(tgfName: String, paramStrs: Seq[String], sc: SharkContext) = { val applyMethodOpt = getMethod(tgfName, "apply") - if (applyMethodOpt.isEmpty) throw new QueryExecutionException("TGF " + tgfName + " needs to implement apply()") + if (applyMethodOpt.isEmpty) { + throw new QueryExecutionException("TGF " + tgfName + " needs to implement apply()") + } + val applyMethod = applyMethodOpt.get val typeNames: Seq[String] = applyMethod.getParameterTypes.toList.map(_.toString) - if (paramStrs.length != typeNames.length) throw new QueryExecutionException("Expecting " + typeNames.length + - " parameters to " + tgfName + ", got " + paramStrs.length) + if (paramStrs.length != typeNames.length) { + throw new QueryExecutionException("Expecting " + typeNames.length + + " parameters to " + tgfName + ", got " + paramStrs.length) + } val params = (paramStrs.toList zip typeNames.toList).map { case (param: String, tpe: String) if (tpe.startsWith("class org.apache.spark.rdd.RDD")) => sc.tableRdd(param) @@ -145,14 +156,14 @@ object TGF { val (sharkSchema, resultArr) = tableNameOpt match { case Some(tableName) => // materialize results - val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) - helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) - (Array[ColumnDesc](), Array[Array[Object]]()) + val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) + helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) + (Array[ColumnDesc](), Array[Array[Object]]()) case None => // return results - val newSchema = schema.map{ case (name, tpe) => new ColumnDesc(name, DataTypes.fromManifest(toManifest(tpe)))} - val res = rdd.collect().map{p => p.map( _.asInstanceOf[Object] ).toArray} - (newSchema.toArray, res) + val newSchema = schema.map{ case (name, tpe) => new ColumnDesc(name, DataTypes.fromManifest(toManifest(tpe)))} + val res = rdd.collect().map{p => p.map( _.asInstanceOf[Object] ).toArray} + (newSchema.toArray, res) } new ResultSet(sharkSchema, resultArr) } @@ -168,7 +179,9 @@ object TGF { else if (tpe == "string") classManifest[java.lang.String] else if (tpe == "timestamp") classManifest[Timestamp] else if (tpe == "date") classManifest[Date] - else throw new QueryExecutionException("Unknown column type specified in schema (" + tpe + ")") + else { + throw new QueryExecutionException("Unknown column type specified in schema (" + tpe + ")") + } } } From 2b4e33e8a3c20c6a8c5279928600766a95dc5d87 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 21 Nov 2013 15:34:58 -0800 Subject: [PATCH 278/331] Simplify Shark table properties. All tables are unified by default and all cached tables are stored at `StorageLevel.MEMORY_AND_DISK`. These are the new"shark.cache" options: - "memory": cache in memory and store on disk. These tables are persistent, and automatically reloaded across Shark session restarts. - "memory_only": cache in memory only. These tables are dropped at the end of a Shark session - "none": Hive table --- src/main/scala/shark/SharkCliDriver.scala | 12 +-- src/main/scala/shark/SharkDriver.scala | 6 +- src/main/scala/shark/SharkServer.scala | 31 +++---- .../scala/shark/api/RDDTableFunctions.scala | 15 +--- .../shark/execution/HadoopTableReader.scala | 19 +++-- .../execution/MemoryStoreSinkOperator.scala | 6 +- .../scala/shark/execution/SharkDDLTask.scala | 40 ++++++--- .../scala/shark/execution/SparkLoadTask.scala | 35 +++----- .../shark/execution/TableScanOperator.scala | 9 ++- .../scala/shark/memstore2/CacheType.scala | 12 +-- .../memstore2/MemoryMetadataManager.scala | 47 ++++------- .../scala/shark/memstore2/MemoryTable.scala | 6 +- .../memstore2/PartitionedMemoryTable.scala | 6 +- .../shark/memstore2/SharkTblProperties.scala | 23 +++--- src/main/scala/shark/memstore2/Table.scala | 5 +- .../scala/shark/memstore2/TableRecovery.scala | 22 +++-- src/main/scala/shark/parse/QueryBlock.scala | 10 +-- .../parse/SharkDDLSemanticAnalyzer.scala | 36 +++------ .../parse/SharkLoadSemanticAnalyzer.scala | 7 +- .../shark/parse/SharkSemanticAnalyzer.scala | 69 +++++++--------- src/main/scala/shark/util/HiveUtils.scala | 8 +- .../scala/shark/util/QueryRewriteUtils.scala | 9 +-- src/test/scala/shark/SQLSuite.scala | 81 +++++++------------ 23 files changed, 219 insertions(+), 295 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 5826d09a..ebd39a0b 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -67,8 +67,8 @@ object SharkCliDriver { } def main(args: Array[String]) { - val hiveArgs = args.filterNot(_.equals("-loadRdds")) - val loadRdds = hiveArgs.length < args.length + val hiveArgs = args.filterNot(_.equals("-skipRddReload")) + val reloadRdds = hiveArgs.length == args.length val oproc = new OptionsProcessor() if (!oproc.process_stage1(hiveArgs)) { System.exit(1) @@ -147,7 +147,7 @@ object SharkCliDriver { Thread.currentThread().setContextClassLoader(loader) } - val cli = new SharkCliDriver(loadRdds) + val cli = new SharkCliDriver(reloadRdds) cli.setHiveVariables(oproc.getHiveVariables()) // Execute -i init files (always in silent mode) @@ -233,7 +233,7 @@ object SharkCliDriver { } -class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper { +class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelper { private val ss = SessionState.get().asInstanceOf[CliSessionState] @@ -251,8 +251,8 @@ class SharkCliDriver(loadRdds: Boolean = false) extends CliDriver with LogHelper SharkEnv.init() } - if (loadRdds) { - TableRecovery.loadUnifiedViews(processCmd(_)) + if (reloadRdds) { + TableRecovery.reloadRdds(processCmd(_)) } def this() = this(false) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 3c103c1d..1dd8446f 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -35,8 +35,10 @@ import org.apache.hadoop.util.StringUtils import shark.api.TableRDD import shark.api.QueryExecutionException -import shark.execution.{SharkDDLTask, SharkDDLWork, SharkExplainTask, SharkExplainWork, - SparkLoadWork, SparkLoadTask, SparkTask, SparkWork} +import shark.execution.{SharkDDLTask, SharkDDLWork} +import shark.execution.{SharkExplainTask, SharkExplainWork} +import shark.execution.{SparkLoadWork, SparkLoadTask} +import shark.execution.{SparkTask, SparkWork} import shark.memstore2.ColumnarSerDe import shark.parse.{QueryContext, SharkSemanticAnalyzerFactory} import shark.util.QueryRewriteUtils diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index d112960d..7b9bcbf7 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -72,19 +72,19 @@ object SharkServer extends LogHelper { def main(args: Array[String]) { - val cli = new SharkServerCliOptions - cli.parse(args) + val cliOptions = new SharkServerCliOptions + cliOptions.parse(args) // From Hive: It is critical to do this prior to initializing log4j, otherwise // any log specific settings via hiveconf will be ignored. - val hiveconf: Properties = cli.addHiveconfToSystemProperties() + val hiveconf: Properties = cliOptions.addHiveconfToSystemProperties() // From Hive: It is critical to do this here so that log4j is reinitialized // before any of the other core hive classes are loaded LogUtils.initHiveLog4j() val latch = new CountDownLatch(1) - serverTransport = new TServerSocket(cli.port) + serverTransport = new TServerSocket(cliOptions.port) val hfactory = new ThriftHiveProcessorFactory(null, new HiveConf()) { override def getProcessor(t: TTransport) = @@ -92,8 +92,8 @@ object SharkServer extends LogHelper { } val ttServerArgs = new TThreadPoolServer.Args(serverTransport) .processorFactory(hfactory) - .minWorkerThreads(cli.minWorkerThreads) - .maxWorkerThreads(cli.maxWorkerThreads) + .minWorkerThreads(cliOptions.minWorkerThreads) + .maxWorkerThreads(cliOptions.maxWorkerThreads) .transportFactory(new TTransportFactory()) .protocolFactory(new TBinaryProtocol.Factory()) server = new TThreadPoolServer(ttServerArgs) @@ -112,12 +112,13 @@ object SharkServer extends LogHelper { } ) - // Optionally load the cached tables. - execLoadRdds(cli.loadRdds, latch) + // Optionally reload cached tables from a previous session. + execLoadRdds(cliOptions.reloadRdds, latch) // Start serving. - val startupMsg = "Starting Shark server on port " + cli.port + " with " + cli.minWorkerThreads + - " min worker threads and " + cli.maxWorkerThreads + " max worker threads" + val startupMsg = "Starting Shark server on port " + cliOptions.port + " with " + + cliOptions.minWorkerThreads + " min worker threads and " + cliOptions.maxWorkerThreads + + " max worker threads." logInfo(startupMsg) println(startupMsg) server.serve() @@ -138,7 +139,7 @@ object SharkServer extends LogHelper { while (!server.isServing()) {} try { val sshandler = new SharkServerHandler - TableRecovery.loadUnifiedViews(sshandler.execute(_)) + TableRecovery.reloadRdds(sshandler.execute(_)) } catch { case (e: Exception) => logWarning("Unable to load RDDs upon startup", e) } finally { @@ -149,14 +150,14 @@ object SharkServer extends LogHelper { // Used to parse command line arguments for the server. class SharkServerCliOptions extends HiveServerCli { - var loadRdds = false + var reloadRdds = false - val OPTION_LOAD_RDDS = "loadRdds" - OPTIONS.addOption(OptionBuilder.create(OPTION_LOAD_RDDS)) + val OPTION_SKIP_RELOAD_RDDS = "skipRddReload" + OPTIONS.addOption(OptionBuilder.create(OPTION_SKIP_RELOAD_RDDS)) override def parse(args: Array[String]) { super.parse(args) - loadRdds = commandLine.hasOption(OPTION_LOAD_RDDS) + reloadRdds = !commandLine.hasOption(OPTION_SKIP_RELOAD_RDDS) } } } diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 9eeaa3a2..5d2ce93f 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -30,11 +30,7 @@ import shark.util.HiveUtils class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { - def saveAsTable( - tableName: String, - fields: Seq[String], - unifyView: Boolean = false, - reloadOnRestart: Boolean = false): Boolean = { + def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { require(fields.size == this.manifests.size, "Number of column names != number of fields in the RDD.") @@ -60,19 +56,14 @@ class RDDTableFunctions(self: RDD[Product], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() - var isSucessfulCreateTable = HiveUtils.createTableInHive( - tableName, - fields, - manifests, - unifyView, - reloadOnRestart) + var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. val databaseName = Hive.get(SharkContext.hiveconf).getCurrentDatabase() if (isSucessfulCreateTable) { // Create an entry in the MemoryMetadataManager. val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, CacheType.HEAP, unifyView, reloadOnRestart) + databaseName, tableName, CacheType.MEMORY) newTable.tableRDD = rdd try { // Force evaluate to put the data in memory. diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index f78eb51d..31aa84a5 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -59,22 +59,22 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = makeRDDForTable( hiveTable, - None /* fitlerOpt */, - _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]]) + _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + filterOpt = None) /** * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed * RDD that contains deserialized rows. * * @param hiveTable Hive metadata for the table being scanned. + * @param deserializerClass Class of the SerDe used to deserialize Writables read from Hadoop. * @param filterOpt If defined, then the filter is used to reject files contained in the data * directory being read. If None, then all files are accepted. - * @param deserializerClass Class of the SerDe used to deserialize Writables read from Hadoop. */ def makeRDDForTable( hiveTable: HiveTable, - filterOpt: Option[PathFilter], - deserializerClass: Class[_ <: Deserializer]): RDD[_] = { + deserializerClass: Class[_ <: Deserializer], + filterOpt: Option[PathFilter]): RDD[_] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") @@ -107,11 +107,10 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf deserializedHadoopRDD } - override def makeRDDForPartitionedTable( - partitions: Seq[HivePartition]): RDD[_] = { + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { val partitionToDeserializer = partitions.map(part => (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap - makeRDDForPartitionedTable(partitionToDeserializer, None /* filterOpt */) + makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) } /** @@ -119,8 +118,8 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf * tables, a data directory is created for each partition corresponding to keys specified using * 'PARTITION BY'. * - * @param partitionToDeserializer Mapping from a Hive partition metadata object to the SerDe - * class to use to deserialize input Writables. + * @param partitionToDeserializer Mapping from a Hive Partition metadata object to the SerDe + * class to use to deserialize input Writables from the corresponding partition. * @param filterOpt If defined, then the filter is used to reject files contained in the data * subdirectory of each partition being read. If None, then all files are accepted. */ diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 622320c7..cc23c73b 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -194,11 +194,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, - tableName, - cacheMode, - unifyView = false, - reloadOnRestart = false)) + databaseName, tableName, cacheMode)) memoryTable.tableRDD = outputRDD } diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index 9ffa1701..df3e9157 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -75,7 +75,15 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] return 0 } - /** Handles a CREATE TABLE or CTAS. */ + /** + * Updates Shark metastore for a CREATE TABLE or CTAS command. + * + * @param hiveMetadataDb Namespace of the table to create. + * @param createTblDesc Hive metadata object that contains fields needed to create a Shark Table + * entry. + * @param cacheMode How the created table should be stored and maintained (e.g, MEMORY means that + * table data will be in memory and persistent across Shark sessions). + */ def createTable( hiveMetadataDb: Hive, createTblDesc: CreateTableDesc, @@ -83,18 +91,16 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val dbName = hiveMetadataDb.getCurrentDatabase val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps - val unifyView = tblProps.get(SharkTblProperties.UNIFY_VIEW_FLAG.varname).toBoolean - val reloadOnRestart = tblProps.get(SharkTblProperties.RELOAD_ON_RESTART_FLAG.varname).toBoolean val isHivePartitioned = (createTblDesc.getPartCols.size > 0) if (isHivePartitioned) { // Add a new PartitionedMemoryTable entry in the Shark metastore. // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( - dbName, tableName, cacheMode, unifyView, reloadOnRestart, tblProps) + dbName, tableName, cacheMode, tblProps) } else { val memoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( - dbName, tableName, cacheMode, unifyView, reloadOnRestart) + dbName, tableName, cacheMode) // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. memoryTable.tableRDD = new EmptyRDD(SharkEnv.sc) } @@ -102,7 +108,12 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] SharkEnv.memoryMetadataManager.putStats(dbName, tableName, Map[Int, TablePartitionStats]()) } - /** Handles an ALTER TABLE ADD PARTITION. */ + /** + * Updates Shark metastore for an ALTER TABLE ADD PARTITION command. + * + * @param hiveMetadataDb Namespace of the table to update. + * @param addPartitionDesc Hive metadata object that contains fields about the new partition. + */ def addPartition( hiveMetadataDb: Hive, addPartitionDesc: AddPartitionDesc) { @@ -120,8 +131,12 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } /** - * A DropTableDesc is used for both dropping entire tables (i.e., DROP TABLE) and for dropping - * individual partitions of a table (i.e., ALTER TABLE DROP PARTITION). + * Updates Shark metastore when dropping a table or partition. + * + * @param hiveMetadataDb Namespace of the table to drop, or the table that a partition belongs to. + * @param dropTableDesc Hive metadata object used for both dropping entire tables + * (i.e., DROP TABLE) and for dropping individual partitions of a table + * (i.e., ALTER TABLE DROP PARTITION). */ def dropTableOrPartition( hiveMetadataDb: Hive, @@ -148,7 +163,14 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] } } - /** Handles miscellaneous ALTER TABLE 'tableName' commands. */ + /** + * Updates Shark metastore for miscellaneous ALTER TABLE commands. + * + * @param hiveMetadataDb Namespace of the table to update. + * @param alterTableDesc Hive metadata object containing fields needed to handle various table + * update commands, such as ALTER TABLE
RENAME TO. + * + */ def alterTable( hiveMetadataDb: Hive, alterTableDesc: AlterTableDesc) { diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index be038c66..ca919b26 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -51,10 +51,10 @@ import shark.util.HiveUtils * * @param databaseName Namespace for the table being handled. * @param tableName Name of the table being handled. - * @param commandType Type (an enum) of command that will be executed for the target table. See + * @param commandType Enum representing the command that will be executed for the target table. See * SparkLoadWork.CommandTypes for a description of which SQL commands correspond to each type. * @param cacheMode Cache type that the RDD should be stored in (e.g., Spark heap). - * TODO(harvey): Support Tachyon. + * TODO(harvey): Support Tachyon. */ private[shark] class SparkLoadWork( @@ -64,17 +64,6 @@ class SparkLoadWork( val cacheMode: CacheType.CacheType) extends Serializable { - // Used for CommandTypes.NEW_ENTRY. - // True if the table should be stored both on disk and in memory. - var unifyView: Boolean = _ - - // Used for CommandTypes.NEW_ENTRY - // True if the Shark table entry created should be marked as recoverable. Table properties needed - // reload the table across sessions will be preserved by the server shutdown hook. See - // MemoryMetadataManager#processTablesOnShutdown() for shutdown handling and - // TblProperties#removeSharkProperties() for the properties that are preserved. - var reloadOnRestart: Boolean = _ - // Defined if the command is an INSERT and under these conditions: // - Table is partitioned, and the partition being updated already exists // (i.e., `partSpecOpt.isDefined == true`) @@ -100,7 +89,7 @@ object SparkLoadWork { // Type of commands executed by the SparkLoadTask created from a SparkLoadWork. // Corresponding SQL commands for each enum: // - NEW_ENTRY: - // CACHE or ALTER TABLE
SET TBLPROPERTIES('shark.cache' = true ... ) + // CACHE or ALTER TABLE
SET TBLPROPERTIES('shark.cache' = `true` ... ) // - INSERT: // INSERT INTO TABLE
or LOAD DATA INPATH '...' INTO
// - OVERWRITE: @@ -236,12 +225,11 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe case SparkLoadWork.CommandTypes.NEW_ENTRY => { // This is a new entry, e.g. we are caching a new table or partition. // Create a new MemoryTable object and return that. - SharkEnv.memoryMetadataManager.createMemoryTable(databaseName, tableName, work.cacheMode, - work.unifyView, work.reloadOnRestart) + SharkEnv.memoryMetadataManager.createMemoryTable(databaseName, tableName, work.cacheMode) } case _ => { - // This is an existing entry (e.g. we are doing insert or insert overwrite). - // Get the MemoryTable object from the metadata manager. + // This is an existing entry (e.g. we are handling an INSERT or INSERT OVERWRITE). + // Get the MemoryTable object from the Shark metastore. val tableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) assert(tableOpt.exists(_.isInstanceOf[MemoryTable]), "Memory table being updated cannot be found in the Shark metastore.") @@ -271,10 +259,7 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val serDe = hiveTable.getDeserializer serDe.initialize(conf, tableSchema) // Scan the Hive table's data directory. - val inputRDD = hadoopReader.makeRDDForTable( - hiveTable, - pathFilterOpt, - serDe.getClass) + val inputRDD = hadoopReader.makeRDDForTable(hiveTable, serDe.getClass, pathFilterOpt) // Transform the HadoopRDD to an RDD[TablePartition]. val (tablePartitionRDD, tableStats) = materialize( inputRDD, @@ -316,8 +301,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe databaseName, tableName, work.cacheMode, - work.unifyView, - work.reloadOnRestart, hiveTable.getParameters) } case _ => { @@ -410,8 +393,8 @@ object SparkLoadTask { * columns appended to respective row metadata properties. */ private def addPartitionInfoToSerDeProps( - partCols: Seq[String], - baseSerDeProps: Properties): Properties = { + partCols: Seq[String], + baseSerDeProps: Properties): Properties = { val serDeProps = new Properties(baseSerDeProps) // Column names specified by the Constants.LIST_COLUMNS key are delimited by ",". diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 37d87c0e..18d20c39 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -110,13 +110,16 @@ class TableScanOperator extends TopOperator[TableScanDesc] { val tableName = tableNameSplit(1) // There are three places we can load the table from. - // 1. Tachyon table - // 2. Spark heap (block manager), accessed through the Shark MemoryMetadataManager + // 1. Spark heap (block manager), accessed through the Shark MemoryMetadataManager + // 2. Tachyon table // 3. Hive table on HDFS (or other Hadoop storage) val cacheMode = CacheType.fromString( tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. - if (cacheMode == CacheType.HEAP) { + if (isInMemoryTableScan) { + assert (cacheMode == CacheType.MEMORY || cacheMode == CacheType.MEMORY_ONLY, + "Table %s.%s is in Shark metastore, but its cacheMode (%s) indicates otherwise". + format(databaseName, tableName, cacheMode)) val tableReader = new HeapTableReader(tableDesc) if (table.isPartitioned) { return tableReader.makeRDDForPartitionedTable(parts) diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala index b3117c4f..6115c542 100644 --- a/src/main/scala/shark/memstore2/CacheType.scala +++ b/src/main/scala/shark/memstore2/CacheType.scala @@ -24,13 +24,15 @@ object CacheType extends Enumeration { /* * The CacheTypes: - * - NONE: On-disk storage (e.g. a Hive table that is stored in HDFS ). - * - HEAP: refers to Spark's block manager, which coordinates in-memory and on-disk RDD storage. + * - MEMORY: Stored in memory and on disk (i.e., cache is write-through). Persistent across Shark + * sessions. By default, all such tables are reloaded into memory on restart. + * - MEMORY_ONLY: Stored only in memory and dropped at the end of each Shark session. * - TACHYON: A distributed storage system that manages an in-memory cache for sharing files and - * RDDs across cluster frameworks. + RDDs across cluster frameworks. + * - NONE: Stored on disk (e.g., HDFS) and managed by Hive. */ type CacheType = Value - val NONE, HEAP, TACHYON = Value + val MEMORY, MEMORY_ONLY, TACHYON, NONE = Value def shouldCache(c: CacheType): Boolean = (c != NONE) @@ -39,7 +41,7 @@ object CacheType extends Enumeration { if (name == null || name == "" || name.toLowerCase == "false") { NONE } else if (name.toLowerCase == "true") { - HEAP + MEMORY } else { try { // Try to use Scala's Enumeration::withName() to interpret 'name'. diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 2b751f9b..adac9a01 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -73,17 +73,9 @@ class MemoryMetadataManager { def createMemoryTable( databaseName: String, tableName: String, - cacheMode: CacheType.CacheType, - unifyView: Boolean, - reloadOnRestart: Boolean - ): MemoryTable = { + cacheMode: CacheType.CacheType): MemoryTable = { val tableKey = makeTableKey(databaseName, tableName) - val newTable = new MemoryTable( - databaseName, - tableName, - cacheMode, - unifyView, - reloadOnRestart) + val newTable = new MemoryTable(databaseName, tableName, cacheMode) _tables.put(tableKey, newTable) newTable } @@ -92,17 +84,10 @@ class MemoryMetadataManager { databaseName: String, tableName: String, cacheMode: CacheType.CacheType, - unifyView: Boolean, - reloadOnRestart: Boolean, tblProps: JavaMap[String, String] ): PartitionedMemoryTable = { val tableKey = makeTableKey(databaseName, tableName) - val newTable = new PartitionedMemoryTable( - databaseName, - tableName, - cacheMode, - unifyView, - reloadOnRestart) + val newTable = new PartitionedMemoryTable(databaseName, tableName, cacheMode) // Determine the cache policy to use and read any user-specified cache settings. val cachePolicyStr = tblProps.getOrElse(SharkTblProperties.CACHE_POLICY.varname, SharkTblProperties.CACHE_POLICY.defaultVal) @@ -174,30 +159,32 @@ class MemoryMetadataManager { def shutdown() { val db = Hive.get() for (table <- _tables.values) { - if (table.unifyView) { - dropUnifiedView(db, table.databaseName, table.tableName, table.reloadOnRestart) - } else { - // XXXX: Why are we dropping Hive tables? - HiveUtils.dropTableInHive(table.tableName, db.getConf) + table.cacheMode match { + case CacheType.MEMORY => { + dropTableFromMemory(db, table.databaseName, table.tableName) + } + case CacheType.MEMORY_ONLY => HiveUtils.dropTableInHive(table.tableName, db.getConf) + case _ => { + // No need to handle Hive or Tachyon tables, which are persistent and managed by their + // respective systems. + Unit + } } } } /** - * Removes Shark table properties and drops a unified view from the Shark cache. However, if - * `preserveRecoveryProps` is true, then Shark properties needed for table recovery won't be - * removed. + * Drops a table from the Shark cache. However, Shark properties needed for table recovery + * (see TableRecovery#reloadRdds()) won't be removed. * After this method completes, the table can still be scanned from disk. */ - def dropUnifiedView( + def dropTableFromMemory( db: Hive, databaseName: String, - tableName: String, - preserveRecoveryProps: Boolean = false) { + tableName: String) { getTable(databaseName, tableName).foreach { sharkTable => db.setCurrentDatabase(databaseName) val hiveTable = db.getTable(databaseName, tableName) - SharkTblProperties.removeSharkProperties(hiveTable.getParameters, preserveRecoveryProps) // Refresh the Hive `db`. db.alterTable(tableName, hiveTable) // Unpersist the table's RDDs from memory. diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index 5ba3c7ca..e82e2fa1 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -25,10 +25,8 @@ import org.apache.spark.rdd.RDD private[shark] class MemoryTable( databaseName: String, tableName: String, - cacheMode: CacheType.CacheType, - unifiedView: Boolean, - reloadOnRestart: Boolean) - extends Table(databaseName, tableName, cacheMode, unifiedView, reloadOnRestart) { + cacheMode: CacheType.CacheType) + extends Table(databaseName, tableName, cacheMode) { // RDD that contains the contents of this table. private var _tableRDD: RDD[TablePartition] = _ diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index f8729225..b649d84d 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -39,10 +39,8 @@ private[shark] class PartitionedMemoryTable( databaseName: String, tableName: String, - cacheMode: CacheType.CacheType, - unifiedView: Boolean, - reloadOnRestart: Boolean) - extends Table(databaseName, tableName, cacheMode, unifiedView, reloadOnRestart) { + cacheMode: CacheType.CacheType) + extends Table(databaseName, tableName, cacheMode) { /** * A simple, mutable wrapper for an RDD. This is needed so that a entry maintained by a diff --git a/src/main/scala/shark/memstore2/SharkTblProperties.scala b/src/main/scala/shark/memstore2/SharkTblProperties.scala index f044317c..befc91d1 100644 --- a/src/main/scala/shark/memstore2/SharkTblProperties.scala +++ b/src/main/scala/shark/memstore2/SharkTblProperties.scala @@ -36,12 +36,6 @@ object SharkTblProperties { // "shark.cache.partition.cachePolicy" property above. val MAX_PARTITION_CACHE_SIZE = new TableProperty("shark.cache.policy.maxSize", "10") - // Default value for the "shark.cache.unify" table property. - val UNIFY_VIEW_FLAG = new TableProperty("shark.cache.unifyView", "true") - - // Default value for the "shark.cache.reloadOnRestart" table property. - val RELOAD_ON_RESTART_FLAG = new TableProperty("shark.cache.reloadOnRestart", "true") - // Default value for the "shark.cache" table property val CACHE_FLAG = new TableProperty("shark.cache", "true") @@ -56,18 +50,19 @@ object SharkTblProperties { * Returns value for the `variable` table property. If a value isn't present in `tblProps`, then * the default for `variable` will be returned. */ - def initializeWithDefaults(tblProps: JavaMap[String, String]): JavaMap[String, String] = { + def initializeWithDefaults( + tblProps: JavaMap[String, String], + isPartitioned: Boolean = false): JavaMap[String, String] = { tblProps.put(CACHE_FLAG.varname, CACHE_FLAG.defaultVal) - tblProps.put(UNIFY_VIEW_FLAG.varname, UNIFY_VIEW_FLAG.defaultVal) - tblProps.put(RELOAD_ON_RESTART_FLAG.varname, RELOAD_ON_RESTART_FLAG.defaultVal) + if (isPartitioned) { + tblProps.put(CACHE_POLICY.varname, CACHE_POLICY.defaultVal) + } tblProps } - def removeSharkProperties(tblProps: JavaMap[String, String], preserveRecoveryProps: Boolean) { + def removeSharkProperties(tblProps: JavaMap[String, String]) { tblProps.remove(CACHE_FLAG.varname) - tblProps.remove(UNIFY_VIEW_FLAG.varname) - if (!preserveRecoveryProps) { - tblProps.remove(RELOAD_ON_RESTART_FLAG.varname) - } + tblProps.remove(CACHE_POLICY.varname) + tblProps.remove(MAX_PARTITION_CACHE_SIZE.varname) } } diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 88035e4f..8cb573f2 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -21,12 +21,11 @@ package shark.memstore2 * A container for table metadata managed by Shark and Spark. Subclasses are responsible for * how RDDs are set, stored, and accessed. * + * @param databaseName Namespace for this table. * @param tableName Name of this table. * @param cacheMode Type of memory storage used for the table (e.g., the Spark block manager). */ private[shark] abstract class Table( var databaseName: String, var tableName: String, - var cacheMode: CacheType.CacheType, - var unifyView: Boolean, - var reloadOnRestart: Boolean) + var cacheMode: CacheType.CacheType) diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala index deeae40a..d671aa8d 100644 --- a/src/main/scala/shark/memstore2/TableRecovery.scala +++ b/src/main/scala/shark/memstore2/TableRecovery.scala @@ -32,26 +32,22 @@ object TableRecovery extends LogHelper { val db = Hive.get() /** - * Loads any tables with the `SharkTblProperties.RELOAD_ON_RESTART_FLAG` property set to true. + * Loads any cached tables with MEMORY as its `shark.cache` property. * @param cmdRunner The runner that is responsible for taking a cached table query and * a) Creating the table metadata in Hive Meta Store * b) Loading the table as an RDD in memory * @see SharkServer for an example usage. */ - def loadUnifiedViews(cmdRunner: String => Unit) { + def reloadRdds(cmdRunner: String => Unit) { // Filter for tables that should be reloaded into the cache. val currentDbName = db.getCurrentDatabase() - for (databaseName <- db.getAllDatabases()) { - for (tableName <- db.getAllTables(databaseName)) { - val tblProps = db.getTable(databaseName, tableName).getParameters - val shouldReload = Option(tblProps.get(SharkTblProperties.RELOAD_ON_RESTART_FLAG.varname)). - exists(_.toBoolean) - if (shouldReload) { - logInfo("Reloading %s.%s into memory.".format(databaseName, tableName)) - // TODO(harvey): Executing SQL directly is a bit of a cop-out... - val cmd = QueryRewriteUtils.cacheToAlterTable("CACHE %s".format(tableName)) - cmdRunner(cmd) - } + for (databaseName <- db.getAllDatabases(); tableName <- db.getAllTables(databaseName)) { + val tblProps = db.getTable(databaseName, tableName).getParameters + val cacheMode = CacheType.fromString(tblProps.get(SharkTblProperties.CACHE_FLAG.varname)) + if (cacheMode == CacheType.MEMORY) { + logInfo("Reloading %s.%s into memory.".format(databaseName, tableName)) + val cmd = QueryRewriteUtils.cacheToAlterTable("CACHE %s".format(tableName)) + cmdRunner(cmd) } } db.setCurrentDatabase(currentDbName) diff --git a/src/main/scala/shark/parse/QueryBlock.scala b/src/main/scala/shark/parse/QueryBlock.scala index 2c99f5da..4d79f12a 100644 --- a/src/main/scala/shark/parse/QueryBlock.scala +++ b/src/main/scala/shark/parse/QueryBlock.scala @@ -32,13 +32,9 @@ import shark.memstore2.CacheType._ class QueryBlock(outerID: String, alias: String, isSubQuery: Boolean) extends HiveQueryBlock(outerID, alias, isSubQuery) { - // The CacheType for the table that will be created from CREATE TABLE/CTAS. - var cacheModeForCreateTable = CacheType.NONE - - var reloadOnRestart: Boolean = false - - // Whether the created to be created or the table specified by CACHED should be backed by disk. - var unifyView = false + // The CacheType for the table that will be created from CREATE TABLE/CTAS, or updated for an + // INSERT. + var cacheMode = CacheType.NONE // Descriptor for the table being updated by an INSERT. var targetTableDesc: TableDesc = _ diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index 46223428..f0a884ca 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -66,19 +66,13 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) * Handle table property changes. * How Shark-specific changes are handled: * - "shark.cache": - * If 'true', then create a SparkLoadTask to load the Hive table into memory. + * If the value evaluated by CacheType#shouldCache() is `true`, then create a SparkLoadTask to + * load the Hive table into memory. * Set it as a dependent of the Hive DDLTask. A SharkDDLTask counterpart isn't created because * the HadoopRDD creation and transformation isn't a direct Shark metastore operation * (unlike the other cases handled in SharkDDLSemantiAnalyzer). * * If 'false', then create a SharkDDLTask that will delete the table entry in the Shark * metastore. - * - * - "shark.cache.unifyView" : - * If 'true' and "shark.cache" is true, then the SparkLoadTask created should read this from the - * table properties when adding an entry to the Shark metastore. - * - * TODO(harvey): Add this, though reevaluate it too...some Spark RDDs might depend on the old - * version of the RDD, so simply dropping it might not work. */ def analyzeAlterTableProperties(ast: ASTNode) { val databaseName = db.getCurrentDatabase() @@ -89,7 +83,8 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) val oldCacheMode = CacheType.fromString(oldTblProps.get(SharkTblProperties.CACHE_FLAG.varname)) val newCacheMode = CacheType.fromString(newTblProps.get(SharkTblProperties.CACHE_FLAG.varname)) - if (!CacheType.shouldCache(oldCacheMode) && CacheType.shouldCache(newCacheMode)) { + val isAlreadyCached = SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName) + if (!isAlreadyCached && newCacheMode == CacheType.MEMORY) { // The table should be cached (and is not already cached). val partSpecsOpt = if (hiveTable.isPartitioned) { val columnNames = hiveTable.getPartCols.map(_.getName) @@ -104,26 +99,21 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) None } newTblProps.put(SharkTblProperties.CACHE_FLAG.varname, newCacheMode.toString) - val unifyView = SharkTblProperties.getOrSetDefault(newTblProps, - SharkTblProperties.UNIFY_VIEW_FLAG).toBoolean - val reloadOnRestart = SharkTblProperties.getOrSetDefault(newTblProps, - SharkTblProperties.RELOAD_ON_RESTART_FLAG).toBoolean - val sparkLoadWork = new SparkLoadWork(databaseName, tableName, - SparkLoadWork.CommandTypes.NEW_ENTRY, newCacheMode) - sparkLoadWork.unifyView = unifyView - sparkLoadWork.reloadOnRestart = reloadOnRestart + val sparkLoadWork = new SparkLoadWork( + databaseName, + tableName, + SparkLoadWork.CommandTypes.NEW_ENTRY, + newCacheMode) partSpecsOpt.foreach(partSpecs => sparkLoadWork.partSpecs = partSpecs) rootTasks.head.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) } if (CacheType.shouldCache(oldCacheMode) && !CacheType.shouldCache(newCacheMode)) { - val isUnifiedView = Option(oldTblProps.get(SharkTblProperties.UNIFY_VIEW_FLAG.varname)). - exists(_.toBoolean) - // Uncache the table. - if (isUnifiedView) { - SharkEnv.memoryMetadataManager.dropUnifiedView(db, databaseName, tableName) + if (oldCacheMode == CacheType.MEMORY) { + // Uncache the table. + SharkEnv.memoryMetadataManager.dropTableFromMemory(db, databaseName, tableName) } else { throw new SemanticException( - "Only unified views can be uncached. A memory-only table should be dropped.") + "A memory-only table should be dropped.") } } diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index 3e2b6f9a..f7a186b0 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -18,13 +18,16 @@ package shark.parse import scala.collection.JavaConversions._ + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{CopyTask, MoveTask, TaskFactory} import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable} import org.apache.hadoop.hive.ql.parse.{ASTNode, BaseSemanticAnalyzer, LoadSemanticAnalyzer} import org.apache.hadoop.hive.ql.plan._ -import shark.execution.SparkLoadWork + import shark.{LogHelper, SharkEnv} +import shark.execution.SparkLoadWork +import shark.memstore2.CacheType class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(conf) { @@ -41,7 +44,7 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con val databaseName = db.getCurrentDatabase() val tableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) - if (tableOpt.exists(_.unifyView)) { + if (tableOpt.exists(table => table.cacheMode == CacheType.MEMORY)) { // Find the arguments needed to instantiate a SparkLoadWork. val tableSpec = new BaseSemanticAnalyzer.tableSpec(db, conf, tableASTNode) val hiveTable = tableSpec.tableHandle diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index 5946ec3e..cdbe8ba3 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -190,11 +190,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.targetTableDesc = tableDesc // If useUnionRDD is true, the sink op is for INSERT INTO. val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) - val cacheMode = table.cacheMode val isPartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( databaseName, cachedTableName) var hivePartitionKey = if (isPartitioned) { - if (cacheMode == CacheType.TACHYON) { + if (table.cacheMode == CacheType.TACHYON) { throw new SemanticException( "Shark does not support caching Hive-partitioned table(s) in Tachyon.") } @@ -202,10 +201,11 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with } else { new String } - if (table.unifyView) { - // The table being updated is a unified view, a SparkLoadTask will be created - // by the genMapRedTasks() call below. Set fields in `qb` that will be needed. - qb.unifyView = true + if (table.cacheMode == CacheType.MEMORY) { + // The table being updated is stored in memory and backed by disk, a + // SparkLoadTask will be created by the genMapRedTasks() call below. Set fields + // in `qb` that will be needed. + qb.cacheMode = table.cacheMode qb.targetTableDesc = tableDesc OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) } else { @@ -215,7 +215,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with databaseName, _resSchema.size, /* numColumns */ hivePartitionKey, - cacheMode, + table.cacheMode, useUnionRDD) } } else { @@ -231,13 +231,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with Seq { // For a single output, we have the option of choosing the output // destination (e.g. CTAS with table property "shark.cache" = "true"). - if (qb.isCTAS && qb.createTableDesc != null && - CacheType.shouldCache(qb.cacheModeForCreateTable)) { - // The table being created from CTAS should be cached. Check whether it should be - // synchronized with disk (i.e., maintain a unified view) or memory-only. + if (qb.isCTAS && qb.createTableDesc != null && CacheType.shouldCache(qb.cacheMode)) { + // The table being created from CTAS should be cached. val tblProps = qb.createTableDesc.getTblProps - // TODO(harvey): Set this during analysis - if (qb.unifyView) { + if (qb.cacheMode == CacheType.MEMORY) { // Save the preferred storage level, since it's needed to create a SparkLoadTask in // genMapRedTasks(). OperatorFactory.createSharkFileOutputPlan(hiveSinkOps.head) @@ -248,7 +245,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.createTableDesc.getDatabaseName, _resSchema.size, /* numColumns */ new String, /* hivePartitionKey */ - qb.cacheModeForCreateTable, + qb.cacheMode, false /* useUnionRDD */) } } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { @@ -361,7 +358,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with */ } - if (qb.unifyView) { + if (qb.cacheMode == CacheType.MEMORY) { // Create a SparkLoadTask used to scan and load disk contents into the cache. val sparkLoadWork = if (qb.isCTAS) { // For cached tables, Shark-specific table properties should be set in @@ -374,9 +371,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with qb.createTableDesc.getDatabaseName, qb.createTableDesc.getTableName, SparkLoadWork.CommandTypes.NEW_ENTRY, - qb.cacheModeForCreateTable) - sparkLoadWork.unifyView = qb.unifyView - sparkLoadWork.reloadOnRestart = qb.reloadOnRestart + qb.cacheMode) sparkLoadWork } else { // Split from 'databaseName.tableName' @@ -474,32 +469,25 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // There are two cases that will enable caching: // 1) Table name includes "_cached" or "_tachyon". // 2) The "shark.cache" table property is "true", or the string representation of a supported - // cache mode (heap, Tachyon). - var cacheMode = CacheType.fromString(createTableProperties.get( - SharkTblProperties.CACHE_FLAG.varname)) - // Continue planning based on the 'cacheMode' read. - if (cacheMode == CacheType.HEAP || (checkTableName && tableName.endsWith("_cached"))) { - cacheMode = CacheType.HEAP - createTableProperties.put(SharkTblProperties.CACHE_FLAG.varname, cacheMode.toString) - } else if (cacheMode == CacheType.TACHYON || - (checkTableName && tableName.endsWith("_tachyon"))) { - cacheMode = CacheType.TACHYON - createTableProperties.put(SharkTblProperties.CACHE_FLAG.varname, cacheMode.toString) + // cache mode (memory, memory-only, Tachyon). + var cacheMode = CacheType.fromString( + createTableProperties.get(SharkTblProperties.CACHE_FLAG.varname)) + if (checkTableName) { + if (tableName.endsWith("_cached")) { + cacheMode = CacheType.MEMORY + } else if (tableName.endsWith("_tachyon")) { + cacheMode = CacheType.TACHYON + } } + // Continue planning based on the 'cacheMode' read. val shouldCache = CacheType.shouldCache(cacheMode) if (shouldCache) { - // Add Shark table properties to the QueryBlock. - queryBlock.cacheModeForCreateTable = cacheMode - queryBlock.unifyView = SharkTblProperties.getOrSetDefault(createTableProperties, - SharkTblProperties.UNIFY_VIEW_FLAG).toBoolean - queryBlock.reloadOnRestart = SharkTblProperties.getOrSetDefault(createTableProperties, - SharkTblProperties.RELOAD_ON_RESTART_FLAG).toBoolean - - if (!queryBlock.unifyView) { - // Directly set the ColumnarSerDe if the table will be stored memory-only. + if (cacheMode == CacheType.MEMORY_ONLY) { + // Directly set the ColumnarSerDe property. createTableDesc.setSerName(classOf[ColumnarSerDe].getName) } + createTableProperties.put(SharkTblProperties.CACHE_FLAG.varname, cacheMode.toString) } // For CTAS ('isRegularCreateTable' is false), the MemoryStoreSinkOperator creates a new @@ -517,11 +505,12 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with hiveDDLTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } - queryBlock.cacheModeForCreateTable = cacheMode + queryBlock.cacheMode = cacheMode queryBlock.setTableDesc(createTableDesc) } - return queryStmtASTNode + queryStmtASTNode } + } diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index c5bf9888..f8d595fb 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -17,7 +17,7 @@ package shark.util -import java.util.{ArrayList => JavaArrayList, Arrays => JavaArrays} +import java.util.{Arrays => JavaArrays, ArrayList => JavaArrayList} import java.util.{HashSet => JHashSet} import java.util.Properties @@ -27,10 +27,10 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.UnionStructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.UnionStructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.exec.DDLTask @@ -90,8 +90,6 @@ private[shark] object HiveUtils { tableName: String, columnNames: Seq[String], columnTypes: Seq[ClassManifest[_]], - unifyView: Boolean = false, - reloadOnRestart: Boolean = false, hiveConf: HiveConf = new HiveConf): Boolean = { val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") diff --git a/src/main/scala/shark/util/QueryRewriteUtils.scala b/src/main/scala/shark/util/QueryRewriteUtils.scala index de2396ba..8ddd6e4a 100644 --- a/src/main/scala/shark/util/QueryRewriteUtils.scala +++ b/src/main/scala/shark/util/QueryRewriteUtils.scala @@ -28,12 +28,7 @@ object QueryRewriteUtils { val cmdSplit = cmd.split(' ') if (cmdSplit.size == 2) { val tableName = cmdSplit(1) - """ - ALTER TABLE %s SET TBLPROPERTIES ( - 'shark.cache' = 'true', - 'shark.cache.unifyView' = 'true', - 'shark.cache.reloadOnRestart' = 'true') - """.format(tableName) + "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache' = 'true')".format(tableName) } else { throw new SemanticException("CACHE accepts a single table name: 'CACHE
'") } @@ -45,7 +40,7 @@ object QueryRewriteUtils { val tableName = cmdSplit(1) "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache' = 'false')".format(tableName) } else { - throw new SemanticException("UNCACHE accepts a single table name: 'CACHE
'") + throw new SemanticException("UNCACHE accepts a single table name: 'UNCACHE
'") } } } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 3734e506..ed190cfe 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.UnionRDD import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException -import shark.memstore2.{MemoryMetadataManager, PartitionedMemoryTable} +import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} class SQLSuite extends FunSuite with BeforeAndAfterAll { @@ -139,8 +139,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { partitioned by (keypart int) tblproperties('shark.cache' = 'true', 'shark.cache.policy.maxSize' = '%d', - 'shark.cache.policy' = '%s', - 'shark.cache.storageLevel' = 'MEMORY_AND_DISK') + 'shark.cache.policy' = '%s') """.format( tableName, maxCacheSize, @@ -168,7 +167,8 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Check that the table is in memory and is a unified view. val sharkTableOpt = sharkMetastore.getTable(DEFAULT_DB_NAME, cachedTableName) assert(sharkTableOpt.isDefined, "Table %s cannot be found in the Shark meatstore") - assert(sharkTableOpt.get.unifyView, "'unifyView' field for table %s is false") + assert(sharkTableOpt.get.cacheMode == CacheType.MEMORY, + "'shark.cache' field for table %s is not CacheType.MEMORY") // Load a non-cached copy of the table into memory. val cacheSum = sc.sql("select sum(key) from %s".format(cachedTableName))(0) @@ -909,64 +909,64 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } ////////////////////////////////////////////////////////////////////////////// - // Creating unified views + // Default cache mode is CacheType.MEMORY (unified view) ////////////////////////////////////////////////////////////////////////////// - test ("Cached table created by CREATE TABLE, with table properties, is unified view by default") { + test ("Table created by CREATE TABLE, with table properties, is CacheType.MEMORY by default") { sc.runSql("drop table if exists test_unify_creation") sc.runSql("""create table test_unify_creation (key int, val string) tblproperties('shark.cache'='true')""") val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation").get - assert(table.unifyView) + assert(table.cacheMode == CacheType.MEMORY) sc.runSql("drop table if exists test_unify_creation") } - test ("Cached table created by CREATE TABLE, with '_cached', is unified view by default") { + test ("Table created by CREATE TABLE, with '_cached', is CacheType.MEMORY by default") { sc.runSql("drop table if exists test_unify_creation_cached") sc.runSql("create table test_unify_creation_cached(key int, val string)") val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation_cached").get - assert(table.unifyView) + assert(table.cacheMode == CacheType.MEMORY) sc.runSql("drop table if exists test_unify_creation_cached") } - test ("Cached table created by CTAS, with table properties, is unified view by default") { + test ("Table created by CTAS, with table properties, is CacheType.MEMORY by default") { sc.runSql("drop table if exists test_unify_ctas") - sc.runSql("""create table test_unify_ctas - tblproperties('shark.cache' = 'true') as select * from test""") + sc.runSql("""create table test_unify_ctas tblproperties('shark.cache' = 'true') + as select * from test""") val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas").get - assert(table.unifyView) + assert(table.cacheMode == CacheType.MEMORY) expectSql("select count(*) from test_unify_ctas", "500") sc.runSql("drop table if exists test_unify_ctas") } - test ("Cached table created by CTAS, with '_cached', is unified view by default") { + test ("Table created by CTAS, with '_cached', is CacheType.MEMORY by default") { sc.runSql("drop table if exists test_unify_ctas_cached") sc.runSql("create table test_unify_ctas_cached as select * from test") val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas_cached").get - assert(table.unifyView) + assert(table.cacheMode == CacheType.MEMORY) expectSql("select count(*) from test_unify_ctas_cached", "500") sc.runSql("drop table if exists test_unify_ctas_cached") } - test ("Don't maintain unified view for CREATE TABLE when 'shark.cache.unifyView' is false") { + test ("CREATE TABLE when 'shark.cache' is CacheType.MEMORY_ONLY") { sc.runSql("drop table if exists test_non_unify_creation") sc.runSql("""create table test_non_unify_creation(key int, val string) - tblproperties('shark.cache' = 'true', 'shark.cache.unifyView' = 'false')""") + tblproperties('shark.cache' = 'memory_only')""") val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_creation").get - assert(!table.unifyView) + assert(table.cacheMode == CacheType.MEMORY_ONLY) sc.runSql("drop table if exists test_non_unify_creation") } - test ("Don't maintiain unified view for CTAS when 'shark.cache.unifyView' is false") { + test ("CTAS when 'shark.cache' is CacheType.MEMORY_ONLY") { sc.runSql("drop table if exists test_non_unify_ctas") sc.runSql("""create table test_non_unify_ctas tblproperties - ('shark.cache' = 'true', 'shark.cache.unifyView' = 'false') as select * from test""") + ('shark.cache' = 'memory_only') as select * from test""") val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_ctas").get - assert(!table.unifyView) + assert(table.cacheMode == CacheType.MEMORY_ONLY) sc.runSql("drop table if exists test_non_unify_ctas") } ////////////////////////////////////////////////////////////////////////////// - // LOAD for unified views + // LOAD for tables cached in memory and stored on disk (unified view) ////////////////////////////////////////////////////////////////////////////// test ("LOAD INTO unified view") { sc.runSql("drop table if exists unified_view_cached") @@ -1014,7 +1014,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { } ////////////////////////////////////////////////////////////////////////////// - // INSERTS for unified views + // INSERT for tables cached in memory and stored on disk (unified view) ////////////////////////////////////////////////////////////////////////////// test ("INSERT INTO unified view") { sc.runSql("drop table if exists unified_view_cached") @@ -1062,8 +1062,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test ("ALTER TABLE caches non-partitioned table if 'shark.cache' is set to true") { sc.runSql("drop table if exists unified_load") sc.runSql("create table unified_load as select * from test") - sc.runSql("""alter table unified_load set - tblproperties('shark.cache' = 'true', 'shark.cache.unifyView' = 'true')""") + sc.runSql("alter table unified_load set tblproperties('shark.cache' = 'true')") expectUnifiedKVTable("unified_load") sc.runSql("drop table if exists unified_load") } @@ -1072,8 +1071,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists unified_part_load") sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") - sc.runSql("""alter table unified_part_load set - tblproperties('shark.cache' = 'true', 'shark.cache.unifyView' = 'true')""") + sc.runSql("alter table unified_part_load set tblproperties('shark.cache' = 'true')") expectUnifiedKVTable("unified_part_load", Some(Map("keypart" -> "1"))) sc.runSql("drop table if exists unified_part_load") } @@ -1081,8 +1079,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { test ("ALTER TABLE uncaches non-partitioned table if 'shark.cache' is set to false") { sc.runSql("drop table if exists unified_load") sc.runSql("create table unified_load as select * from test") - sc.runSql("""alter table unified_load set - tblproperties('shark.cache' = 'false', 'shark.cache.unifyView' = 'false')""") + sc.runSql("alter table unified_load set tblproperties('shark.cache' = 'false')") assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_load")) expectSql("select count(*) from unified_load", "500") sc.runSql("drop table if exists unified_load") @@ -1092,8 +1089,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.runSql("drop table if exists unified_part_load") sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") - sc.runSql("""alter table unified_part_load set - tblproperties('shark.cache' = 'false')""") + sc.runSql("alter table unified_part_load set tblproperties('shark.cache' = 'false')") assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_part_load")) expectSql("select count(*) from unified_part_load", "500") sc.runSql("drop table if exists unified_part_load") @@ -1106,8 +1102,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Double check the table properties. val tableName = "unified_load" val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - assert(hiveTable.getProperty("shark.cache") == "HEAP") - assert(hiveTable.getProperty("shark.cache.unifyView") == "true") + assert(hiveTable.getProperty("shark.cache") == "MEMORY") // Check that the cache and disk contents are synchronized. expectUnifiedKVTable(tableName) sc.runSql("drop table if exists unified_load") @@ -1120,17 +1115,16 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Double check the table properties. val tableName = "unified_load" val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - assert(hiveTable.getProperty("shark.cache") == "HEAP") - assert(hiveTable.getProperty("shark.cache.unifyView") == "true") + assert(hiveTable.getProperty("shark.cache") == "MEMORY") // Check that the cache and disk contents are synchronized. expectUnifiedKVTable(tableName) sc.runSql("drop table if exists unified_load") } ////////////////////////////////////////////////////////////////////////////// - // Unified view persistence + // Cached table persistence ////////////////////////////////////////////////////////////////////////////// - test ("Unified views persist across Shark metastore shutdowns.") { + test ("Cached tables persist across Shark metastore shutdowns.") { val globalCachedTableNames = Seq("test_cached", "test_null_cached", "clicks_cached", "users_cached", "test1_cached") @@ -1150,19 +1144,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { val cachedCount = cachedTableCounts(i) assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) - - // Make sure that some Shark table properties are removed. - val tblProps = hiveTable.getParameters - assert(!tblProps.contains("shark.cache"), - "'shark.cache' table property should be removed.") - assert(!tblProps.contains("shark.cachy.unifyView"), - "'shark.unifyView' table property should be removed.") - - // The tables should be recoverable. - assert(tblProps.contains("shark.cache.storageLevel"), - "'shark.cache.storageLevel' needed for table recovery is missing.") - assert(tblProps.contains("shark.cache.reloadOnRestart"), - "'shark.cache.reloadOnRestart' should be true, since this table is marked for recovery.") } // Finally, reload all tables. loadTables() From 29ccdb08eda1d70b17eadac8d70b786b8f2b16d3 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sun, 24 Nov 2013 15:48:41 -0800 Subject: [PATCH 279/331] Latest TGF --- src/main/scala/shark/tgf/TGF.scala | 56 +++++++++--------------------- 1 file changed, 16 insertions(+), 40 deletions(-) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index f67dc0ce..6bf80f1c 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -27,7 +27,7 @@ import org.apache.spark.rdd.RDD import shark.api._ import shark.SharkContext -case class RDDSchema(rdd: RDD[Seq[_]], schema: Seq[Tuple2[String,String]]) +case class RDDSchema(rdd: RDD[Seq[_]], schema: String) private class TGFParser extends JavaTokenParsers { @@ -89,7 +89,9 @@ object TGF { private def getSchema(tgfOutput: Object, tgfName: String): Tuple2[RDD[Seq[_]], Seq[Tuple2[String,String]]] = { if (tgfOutput.isInstanceOf[RDDSchema]) { val rddSchema = tgfOutput.asInstanceOf[RDDSchema] - (rddSchema.rdd, rddSchema.schema) + val schema = parser.parseAll(parser.schema, rddSchema.schema) + + (rddSchema.rdd, schema.get) } else if (tgfOutput.isInstanceOf[RDD[Product]]) { val applyMethod = getMethod(tgfName, "apply") if (applyMethod == None) { @@ -123,18 +125,26 @@ object TGF { val typeNames: Seq[String] = applyMethod.getParameterTypes.toList.map(_.toString) - if (paramStrs.length != typeNames.length) { + val augParams = if (!typeNames.isEmpty && typeNames.head.startsWith("class shark.SharkContext")) { + Seq("sc") ++ paramStrs + } else { + paramStrs + } + + if (augParams.length != typeNames.length) { throw new QueryExecutionException("Expecting " + typeNames.length + - " parameters to " + tgfName + ", got " + paramStrs.length) + " parameters to " + tgfName + ", got " + augParams.length) } - val params = (paramStrs.toList zip typeNames.toList).map { + val params = (augParams.toList zip typeNames.toList).map { + case (param: String, tpe: String) if (tpe.startsWith("class shark.SharkContext")) => sc case (param: String, tpe: String) if (tpe.startsWith("class org.apache.spark.rdd.RDD")) => sc.tableRdd(param) case (param: String, tpe: String) if (tpe.startsWith("long")) => param.toLong case (param: String, tpe: String) if (tpe.startsWith("int")) => param.toInt case (param: String, tpe: String) if (tpe.startsWith("double")) => param.toDouble case (param: String, tpe: String) if (tpe.startsWith("float")) => param.toFloat - case (param: String, tpe: String) if (tpe.startsWith("class String")) => param + case (param: String, tpe: String) if (tpe.startsWith("class java.lang.String") || + tpe.startsWith("class String")) => param.stripPrefix("\"").stripSuffix("\"") case (param: String, tpe: String) => throw new QueryExecutionException("Expected TGF parameter type: " + tpe + " (" + param + ")") } @@ -184,37 +194,3 @@ object TGF { } } } - -//object NameOfTGF { -// import org.apache.spark.mllib.clustering._ -// -// // TGFs need to implement an apply() method. -// // The TGF has to have an apply function that takes any arbitrary primitive types and any number of RDDs. -// // When a TGF is invoked from Shark, every type of RDD is produced by converting Hive tables to RDDs -// // TGFs need to have a return type that is either an RDD[Product] or RDDSchema -// // The former case requires that the apply method has an annotation of the schema (see below) -// // In the latter case the schema is embedded in the RDDSchema function -// @Schema(spec = "year int, state string, product string, sales double, cluster int") -// def apply(sales: RDD[(Int, String, String, Double)], k: Int): -// RDD[(Int, String, String, Double, Int)] = { -// -// val dataset = sales.map{ case (year, state, product, sales) => Array[Double](sales) } -// val model = KMeans.train(dataset, k, 2, 2) -// -// sales.map{ case (year, state, product, sales) => (year, state, product, sales, model.predict(Array(sales))) } -// } -// // Alternatively, using RDDSchema return time you can dynamically decide the columns and their types -// def apply(sales: RDD[(Int, String, String, Double)], k: Int): -// RDDSchema = { -// -// val dataset = sales.map{ case (year, state, product, sales) => Array[Double](sales) } -// val model = KMeans.train(dataset, k, 2, 2) -// -// val rdd = sales.map{ -// case (year, state, product, sales) => List(year, state, product, sales, model.predict(Array(sales))) -// } -// -// RDDSchema(rdd.asInstanceOf[RDD[Seq[_]]], -// List(("year","int"), ("state", "string"), ("product", "string"), ("sales", "double"), ("cluster", "int"))) -// } -//} From db68c04b35e6fcde6ac52e4345db863cd417f533 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sat, 14 Dec 2013 21:25:26 -0800 Subject: [PATCH 280/331] deleted table_rdd_generators.py --- .../tablerdd/table_rdd_generators.py | 75 ------------------- 1 file changed, 75 deletions(-) delete mode 100755 src/main/resources/tablerdd/table_rdd_generators.py diff --git a/src/main/resources/tablerdd/table_rdd_generators.py b/src/main/resources/tablerdd/table_rdd_generators.py deleted file mode 100755 index 8946c706..00000000 --- a/src/main/resources/tablerdd/table_rdd_generators.py +++ /dev/null @@ -1,75 +0,0 @@ -#!/usr/bin/python -from string import Template -import sys - -## This script generates the 22 functions needed to create sqlToRdd's - -p = sys.stdout - -# e.g. createList(1,3, "T[", "]", ",") gives T[1],T[2],T[3] -def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 80, indent = 0): - res = "" - oneLine = res - for y in range(start,stop+1): - res += prefix + str(y) + suffix - oneLine += prefix + str(y) + suffix - if y != stop: - res += sep - oneLine += sep - if len(oneLine) > newlineAfter: - res += "\n" + " "*indent - oneLine = "" - return res - -### The SparkContext declaration -for x in range(2,23): - - inner = "" - for y in range(1,x+1): - if y % 3 == 1: inner += " " - inner += Template(" row.getPrimitiveGeneric[T$num1]($num2)").substitute(num1=y, num2=y-1) - if y != x: inner += "," - if y % 3 == 0: inner += "\n" - inner += " ) )\n" - - tableClass = Template( -""" -class TableRDD$num[$list](prev: RowRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple$num[$list]](prev) { - def schema = prev.schema - - val tableCols = schema.size - if (tableCols != $num) throw new IllegalArgumentException("Table only has " + tableCols + " columns, expecting $num") - - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) - throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } - - override def getPartitions = prev.getPartitions - - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple$num[$list]] = { - prev.compute(split, context).map( row => - new Tuple$num[$list]( - $innerfatlist - } -} -""").substitute(num = x, list = createList(1, x, "T", "", ", ", indent=4), innerfatlist = inner) - - - p.write(tableClass) - -# The SharkContext declarations -for x in range(2,23): - sqlRddFun = Template( -""" - def sqlRdd[$list1](cmd: String): - RDD[Tuple$num[$list2]] = - new TableRDD$num[$list2](sqlRowRdd(cmd), - Seq($list3)) -""").substitute(num = x, - list1 = createList(1, x, "T", ": M", ", ", 80, 4), - list2 = createList(1, x, "T", sep=", ", indent = 4), - list3 = createList(1, x, "m[T", "]", sep=", ", indent = 10)) - p.write(sqlRddFun) From b6c5cf7909845cd892900db9c7dcdda2734c1b46 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sat, 14 Dec 2013 21:41:45 -0800 Subject: [PATCH 281/331] Undoing small formatting changes. Nothing semantic. --- .../scala/shark/api/RDDTableFunctions.scala | 1 + .../scala/shark/api/TableRDDGenerated.scala | 235 +++++++++--------- 2 files changed, 114 insertions(+), 122 deletions(-) diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 7f33619f..50636c5e 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -20,6 +20,7 @@ package shark.api import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.ql.metadata.Hive + import org.apache.spark.rdd.RDD import shark.{SharkContext, SharkEnv} diff --git a/src/main/scala/shark/api/TableRDDGenerated.scala b/src/main/scala/shark/api/TableRDDGenerated.scala index d37a7408..344ee6a7 100644 --- a/src/main/scala/shark/api/TableRDDGenerated.scala +++ b/src/main/scala/shark/api/TableRDDGenerated.scala @@ -1,4 +1,3 @@ - /* * Copyright (C) 2013 The Regents of The University California. * All rights reserved. @@ -40,7 +39,7 @@ class TableSeqRDD(prev: TableRDD) class TableRDD1[T1](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple1[T1]](prev) { def schema = prev.schema @@ -53,17 +52,17 @@ class TableRDD1[T1](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple1[T1]] = { prev.compute(split, context).map( row => new Tuple1[T1]( - row.getPrimitiveGeneric[T1](0) ) ) + row.getPrimitiveGeneric[T1](0) ) ) } } class TableRDD2[T1, T2](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple2[T1, T2]](prev) { def schema = prev.schema @@ -76,17 +75,17 @@ class TableRDD2[T1, T2](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple2[T1, T2]] = { prev.compute(split, context).map( row => new Tuple2[T1, T2]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) } } class TableRDD3[T1, T2, T3](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple3[T1, T2, T3]](prev) { def schema = prev.schema @@ -99,18 +98,18 @@ class TableRDD3[T1, T2, T3](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple3[T1, T2, T3]] = { prev.compute(split, context).map( row => new Tuple3[T1, T2, T3]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) - ) ) + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) + ) ) } } class TableRDD4[T1, T2, T3, T4](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple4[T1, T2, T3, T4]](prev) { def schema = prev.schema @@ -123,18 +122,18 @@ class TableRDD4[T1, T2, T3, T4](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple4[T1, T2, T3, T4]] = { prev.compute(split, context).map( row => new Tuple4[T1, T2, T3, T4]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3) ) ) } } class TableRDD5[T1, T2, T3, T4, T5](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple5[T1, T2, T3, T4, T5]](prev) { def schema = prev.schema @@ -147,18 +146,18 @@ class TableRDD5[T1, T2, T3, T4, T5](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple5[T1, T2, T3, T4, T5]] = { prev.compute(split, context).map( row => new Tuple5[T1, T2, T3, T4, T5]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4) ) ) } } class TableRDD6[T1, T2, T3, T4, T5, T6](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple6[T1, T2, T3, T4, T5, T6]](prev) { def schema = prev.schema @@ -171,19 +170,19 @@ class TableRDD6[T1, T2, T3, T4, T5, T6](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple6[T1, T2, T3, T4, T5, T6]] = { prev.compute(split, context).map( row => new Tuple6[T1, T2, T3, T4, T5, T6]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5) - ) ) + ) ) } } class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]](prev) { def schema = prev.schema @@ -196,11 +195,11 @@ class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { prev.compute(split, context).map( row => new Tuple7[T1, T2, T3, T4, T5, T6, T7]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6) ) ) @@ -208,7 +207,7 @@ class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, } class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]](prev) { def schema = prev.schema @@ -221,11 +220,11 @@ class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { prev.compute(split, context).map( row => new Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7) ) ) @@ -233,7 +232,7 @@ class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, } class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]](prev) { def schema = prev.schema @@ -246,20 +245,20 @@ class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { prev.compute(split, context).map( row => new Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8) - ) ) + ) ) } } class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]](prev) { def schema = prev.schema @@ -272,11 +271,11 @@ class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { prev.compute(split, context).map( row => new Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9) ) ) @@ -285,7 +284,7 @@ class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, } class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]](prev) { def schema = prev.schema @@ -298,11 +297,11 @@ class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { prev.compute(split, context).map( row => new Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10) ) ) @@ -311,7 +310,7 @@ class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, } class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]](prev) { def schema = prev.schema @@ -324,21 +323,21 @@ class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: TableR override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { prev.compute(split, context).map( row => new Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11) - ) ) + ) ) } } class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]](prev) { def schema = prev.schema @@ -351,11 +350,11 @@ class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: T override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { prev.compute(split, context).map( row => new Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -365,7 +364,7 @@ class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: T } class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]](prev) { def schema = prev.schema @@ -378,11 +377,11 @@ class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](pr override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { prev.compute(split, context).map( row => new Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -392,7 +391,7 @@ class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](pr } class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]](prev) { def schema = prev.schema @@ -405,22 +404,22 @@ class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { prev.compute(split, context).map( row => new Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14) - ) ) + ) ) } } class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + mans: Seq[ClassManifest[_]]) extends RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]](prev) { def schema = prev.schema @@ -433,11 +432,11 @@ class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { prev.compute(split, context).map( row => new Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -447,11 +446,9 @@ class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17]](prev) { +class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { def schema = prev.schema private val tableCols = schema.size @@ -463,13 +460,11 @@ class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { prev.compute(split, context).map( row => - new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -479,11 +474,9 @@ class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18]](prev) { +class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { def schema = prev.schema private val tableCols = schema.size @@ -495,28 +488,26 @@ class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { prev.compute(split, context).map( row => - new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17) - ) ) + ) ) } } -class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19]](prev) { +class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]](prev) { def schema = prev.schema private val tableCols = schema.size @@ -528,13 +519,13 @@ class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]] = { prev.compute(split, context).map( row => - new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -545,11 +536,11 @@ class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20]](prev) { +class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]](prev) { def schema = prev.schema private val tableCols = schema.size @@ -561,13 +552,13 @@ class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]] = { prev.compute(split, context).map( row => - new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -578,11 +569,11 @@ class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21]](prev) { +class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]](prev) { def schema = prev.schema private val tableCols = schema.size @@ -594,29 +585,29 @@ class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]] = { prev.compute(split, context).map( row => - new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20) - ) ) + ) ) } } -class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21, T22](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21, T22]](prev) { +class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, +T19, T20, T21, T22](prev: TableRDD, + mans: Seq[ClassManifest[_]]) + extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]](prev) { def schema = prev.schema private val tableCols = schema.size @@ -628,13 +619,13 @@ class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21, T22]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]] = { prev.compute(split, context).map( row => - new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, - T17, T18, T19, T20, T21, T22]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, + T19, T20, T21, T22]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), From 1493683e128f0fdb89a0038ece1e32698c0f0269 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sun, 15 Dec 2013 18:27:10 -0800 Subject: [PATCH 282/331] Documented the code and refactored it to be simpler --- src/main/scala/shark/SharkContext.scala | 33 ---- src/main/scala/shark/tgf/TGF.scala | 211 ++++++++++++++++-------- 2 files changed, 145 insertions(+), 99 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 809dcde2..3c6cdc1b 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -93,39 +93,6 @@ class SharkContext( } } - /** - * Takes a table name and generates an RDD of tuples of the right size for it, if more than 22 columns it uses Seqs - * @param tableName - * @return RDD of either Tuple or Seq (if the table contains more than 22 columns) - */ - def tableRdd(tableName: String): RDD[_] = { - val rdd = sql2rdd("SELECT * FROM " + tableName) - rdd.schema.size match { - case 2 => new TableRDD2(rdd, Seq()) - case 3 => new TableRDD3(rdd, Seq()) - case 4 => new TableRDD4(rdd, Seq()) - case 5 => new TableRDD5(rdd, Seq()) - case 6 => new TableRDD6(rdd, Seq()) - case 7 => new TableRDD7(rdd, Seq()) - case 8 => new TableRDD8(rdd, Seq()) - case 9 => new TableRDD9(rdd, Seq()) - case 10 => new TableRDD10(rdd, Seq()) - case 11 => new TableRDD11(rdd, Seq()) - case 12 => new TableRDD12(rdd, Seq()) - case 13 => new TableRDD13(rdd, Seq()) - case 14 => new TableRDD14(rdd, Seq()) - case 15 => new TableRDD15(rdd, Seq()) - case 16 => new TableRDD16(rdd, Seq()) - case 17 => new TableRDD17(rdd, Seq()) - case 18 => new TableRDD18(rdd, Seq()) - case 19 => new TableRDD19(rdd, Seq()) - case 20 => new TableRDD20(rdd, Seq()) - case 21 => new TableRDD21(rdd, Seq()) - case 22 => new TableRDD22(rdd, Seq()) - case _ => new TableSeqRDD(rdd) - } - } - /** * Execute a SQL command and return the results as a TableRDD. The SQL command must be * a SELECT statement. diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 6bf80f1c..770d14bc 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -27,49 +27,90 @@ import org.apache.spark.rdd.RDD import shark.api._ import shark.SharkContext -case class RDDSchema(rdd: RDD[Seq[_]], schema: String) - -private class TGFParser extends JavaTokenParsers { - - /* Code to enable case-insensitive modifiers to strings, e.g. "DataBricks".ci will match "databricks" */ - class MyString(str: String) { - def ci: Parser[String] = ("(?i)" + str).r - } - - implicit def stringToRichString(str: String): MyString = new MyString(str) - - def tgf: Parser[Any] = saveTgf | basicTgf +/** + * This object is responsible for handling TGF (Table Generating Function) commands. + * + * -- TGF Commands -- + * generate tgfname(param1, param2, ... , param_n) + * generate tgfname(param1, param2, ... , param_n) save as tablename + * + * Parameters can either be of primitive types, e.g. int, or of type RDD[Product]. + * TGF.execute() will use reflection looking for an object of name "tgfname", invoking apply() with the primitive + * values. If the type of a parameter to apply() is RDD[Product], it will assume the parameter is the name of a table, + * which it will turn into an RDD before invoking apply(). + * + * For example, "generate MyObj(25, emp)" will invoke MyObj.apply(25, sc.sql2rdd("select * from emp")), assuming + * the TGF object (MyObj) has an apply function that takes an int and an RDD[Product]. + * + * The "save as" version of the command saves the output in a new table named "tablename", whereas the other version + * returns a ResultSet + * + * -- Defining TGF objects -- + * TGF objects need to have an apply() function and take an arbitrary number of either primitive or RDD[Product] typed + * parameters. The apply() function should either return an RDD[Product] or RDDSchema. When the former case is used, + * the returned table's schema and column names need to be defined through a Java annotation called @Schema. Here is + * a short example: + * + * object MyTGF1 { + * \@Schema(spec = "name string, age int") + * def apply(table1: RDD[(String, String, Int)]): RDD[Product] = { + * // code that manipulates table1 and returns a new RDD of tuples + * } + * } + * + * Sometimes, the TGF dynamically determines the number or types of columns returned. In this case, the TGF can + * use the RDDSchema return type instead of Java annotations. RDDSchema simply contains a schema string and an RDD + * of results. For example: + * + * object MyTGF2 { + * \@Schema(spec = "name string, age int") + * def apply(table1: RDD[(String, String, Int)]): RDD[Product] = { + * // code that manipulates table1 and creates a result rdd + * return RDDSchema(rdd.asInstanceOf[RDD[Seq[_]]], "name string, age int") + * } + * } + * + * Sometimes the TGF needs to internally make SQL calls. For that, it needs access to a SharkContext object. Therefore, + * def apply(sc: SharkContext, table1: RDD[(String, String, Int)]): RDD[Product] = { + * // code that can use sc, for example by calling sc.sql2rdd() + * // code that manipulates table1 and returns a new RDD of tuples + * } + */ - /** - * @return Tuple2 containing a TGF method name and a List of parameters as strings - */ - def basicTgf: Parser[Tuple2[String, List[String]]] = { - ("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ - { case id1 ~ x => (id1, x.asInstanceOf[List[String]]) } - } +object TGF { + private val parser = new TGFParser /** - * @return Tuple3 containing a table name, TGF method name and a List of parameters as strings + * Executes a TGF command and gives back the ResultSet. Mainly to be used from SharkContext (e.g. runSql()) + * @param sql TGF command, e.g. "generate name(params)" + * @param sc SharkContext + * @return ResultSet containing the results of the command */ - def saveTgf: Parser[Tuple3[String, String, List[String]]] = { - (("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("SAVE".ci ~ "AS".ci) ~> - ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } - } - - def schema: Parser[Seq[Tuple2[String,String]]] = repsep(nameType, ",") - - def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } + def execute(sql: String, sc: SharkContext): ResultSet = { + val ast = parser.parseAll(parser.tgf, sql).getOrElse{throw new QueryExecutionException("TGF parse error: "+ sql)} - def param: Parser[Any] = stringLiteral | floatingPointNumber | decimalNumber | ident | - failure("Expected a string, number, or identifier as parameters in TGF") + val (tableNameOpt, tgfName, params) = ast match { + case Tuple2(tgfName, params) => (None, tgfName.asInstanceOf[String], params.asInstanceOf[List[String]]) + case Tuple3(tableName, tgfName, params) => (Some(tableName.asInstanceOf[String]), tgfName.asInstanceOf[String], + params.asInstanceOf[List[String]]) + } - def methodName: Parser[String] = - """[a-zA-Z_][\w\.]*""".r -} + val obj = reflectInvoke(tgfName, params, sc) + val (rdd, schema) = getSchema(obj, tgfName) -object TGF { + val (sharkSchema, resultArr) = tableNameOpt match { + case Some(tableName) => // materialize results + val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) + helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) + (Array[ColumnDesc](), Array[Array[Object]]()) - private val parser = new TGFParser + case None => // return results + val newSchema = schema.map{ case (name, tpe) => new ColumnDesc(name, DataTypes.fromManifest(toManifest(tpe)))} + val res = rdd.collect().map{p => p.map( _.asInstanceOf[Object] ).toArray} + (newSchema.toArray, res) + } + new ResultSet(sharkSchema, resultArr) + } private def getMethod(tgfName: String, methodName: String) = { val tgfClazz = try { @@ -82,10 +123,6 @@ object TGF { if (methods.isEmpty) None else Some(methods(0)) } - // private def isOfType(obj: AnyRef, typeString: String) = { - // obj.getClass.getTy - // } - private def getSchema(tgfOutput: Object, tgfName: String): Tuple2[RDD[Seq[_]], Seq[Tuple2[String,String]]] = { if (tgfOutput.isInstanceOf[RDDSchema]) { val rddSchema = tgfOutput.asInstanceOf[RDDSchema] @@ -138,7 +175,7 @@ object TGF { val params = (augParams.toList zip typeNames.toList).map { case (param: String, tpe: String) if (tpe.startsWith("class shark.SharkContext")) => sc - case (param: String, tpe: String) if (tpe.startsWith("class org.apache.spark.rdd.RDD")) => sc.tableRdd(param) + case (param: String, tpe: String) if (tpe.startsWith("class org.apache.spark.rdd.RDD")) => tableRdd(sc, param) case (param: String, tpe: String) if (tpe.startsWith("long")) => param.toLong case (param: String, tpe: String) if (tpe.startsWith("int")) => param.toInt case (param: String, tpe: String) if (tpe.startsWith("double")) => param.toDouble @@ -152,32 +189,6 @@ object TGF { applyMethod.invoke(null, params.asInstanceOf[List[Object]]:_*) } - def execute(sql: String, sc: SharkContext): ResultSet = { - val ast = parser.parseAll(parser.tgf, sql).getOrElse{throw new QueryExecutionException("TGF parse error: "+ sql)} - - val (tableNameOpt, tgfName, params) = ast match { - case Tuple2(tgfName, params) => (None, tgfName.asInstanceOf[String], params.asInstanceOf[List[String]]) - case Tuple3(tableName, tgfName, params) => (Some(tableName.asInstanceOf[String]), tgfName.asInstanceOf[String], - params.asInstanceOf[List[String]]) - } - - val obj = reflectInvoke(tgfName, params, sc) - val (rdd, schema) = getSchema(obj, tgfName) - - val (sharkSchema, resultArr) = tableNameOpt match { - case Some(tableName) => // materialize results - val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) - helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) - (Array[ColumnDesc](), Array[Array[Object]]()) - - case None => // return results - val newSchema = schema.map{ case (name, tpe) => new ColumnDesc(name, DataTypes.fromManifest(toManifest(tpe)))} - val res = rdd.collect().map{p => p.map( _.asInstanceOf[Object] ).toArray} - (newSchema.toArray, res) - } - new ResultSet(sharkSchema, resultArr) - } - private def toManifest(tpe: String): ClassManifest[_] = { if (tpe == "boolean") classManifest[java.lang.Boolean] else if (tpe == "tinyint") classManifest[java.lang.Byte] @@ -193,4 +204,72 @@ object TGF { throw new QueryExecutionException("Unknown column type specified in schema (" + tpe + ")") } } + + def tableRdd(sc: SharkContext, tableName: String): RDD[_] = { + val rdd = sc.sql2rdd("SELECT * FROM " + tableName) + rdd.schema.size match { + case 2 => new TableRDD2(rdd, Seq()) + case 3 => new TableRDD3(rdd, Seq()) + case 4 => new TableRDD4(rdd, Seq()) + case 5 => new TableRDD5(rdd, Seq()) + case 6 => new TableRDD6(rdd, Seq()) + case 7 => new TableRDD7(rdd, Seq()) + case 8 => new TableRDD8(rdd, Seq()) + case 9 => new TableRDD9(rdd, Seq()) + case 10 => new TableRDD10(rdd, Seq()) + case 11 => new TableRDD11(rdd, Seq()) + case 12 => new TableRDD12(rdd, Seq()) + case 13 => new TableRDD13(rdd, Seq()) + case 14 => new TableRDD14(rdd, Seq()) + case 15 => new TableRDD15(rdd, Seq()) + case 16 => new TableRDD16(rdd, Seq()) + case 17 => new TableRDD17(rdd, Seq()) + case 18 => new TableRDD18(rdd, Seq()) + case 19 => new TableRDD19(rdd, Seq()) + case 20 => new TableRDD20(rdd, Seq()) + case 21 => new TableRDD21(rdd, Seq()) + case 22 => new TableRDD22(rdd, Seq()) + case _ => new TableSeqRDD(rdd) + } + } +} + +case class RDDSchema(rdd: RDD[Seq[_]], schema: String) + +private class TGFParser extends JavaTokenParsers { + + /* Code to enable case-insensitive modifiers to strings, e.g. "DataBricks".ci will match "databricks" */ + class MyString(str: String) { + def ci: Parser[String] = ("(?i)" + str).r + } + + implicit def stringToRichString(str: String): MyString = new MyString(str) + + def tgf: Parser[Any] = saveTgf | basicTgf + + /** + * @return Tuple2 containing a TGF method name and a List of parameters as strings + */ + def basicTgf: Parser[Tuple2[String, List[String]]] = { + ("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ + { case id1 ~ x => (id1, x.asInstanceOf[List[String]]) } + } + + /** + * @return Tuple3 containing a table name, TGF method name and a List of parameters as strings + */ + def saveTgf: Parser[Tuple3[String, String, List[String]]] = { + (("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("SAVE".ci ~ "AS".ci) ~> + ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } + } + + def schema: Parser[Seq[Tuple2[String,String]]] = repsep(nameType, ",") + + def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } + + def param: Parser[Any] = stringLiteral | floatingPointNumber | decimalNumber | ident | + failure("Expected a string, number, or identifier as parameters in TGF") + + def methodName: Parser[String] = + """[a-zA-Z_][\w\.]*""".r } From 31b06373c91097a477914622598810d096c5870e Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sun, 15 Dec 2013 20:01:03 -0800 Subject: [PATCH 283/331] Added some simple test cases for TGFs --- src/main/scala/shark/tgf/TGF.scala | 6 +++--- src/test/scala/shark/SQLSuite.scala | 30 ++++++++++++++++++++++++++++- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 770d14bc..8b080695 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -135,12 +135,12 @@ object TGF { throw new QueryExecutionException("TGF lacking apply() method") } - val annotations = applyMethod.get.getAnnotation(classOf[Schema]).spec() - if (annotations == None) { + val annotations = applyMethod.get.getAnnotation(classOf[Schema]) + if (annotations == null || annotations.spec() == null) { throw new QueryExecutionException("No schema annotation found for TGF") } - val schema = parser.parseAll(parser.schema, annotations) + val schema = parser.parseAll(parser.schema, annotations.spec()) if (schema == None) { throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)") } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index ed190cfe..5fb3428f 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -26,12 +26,13 @@ import org.scalatest.FunSuite import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} - +import shark.tgf.{RDDSchema, Schema} class SQLSuite extends FunSuite with BeforeAndAfterAll { @@ -1148,4 +1149,31 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { // Finally, reload all tables. loadTables() } + + ////////////////////////////////////////////////////////////////////////////// + // Table Generating Functions (TGFs) + ////////////////////////////////////////////////////////////////////////////// + + test("Simple TGFs") { + expectSql("generate shark.TestTGF1(test, 15)", Array(15,15,15,17,19).map(_.toString).toArray ) + } + + test("Advanced TGFs with SharkContext and dynamic schemas") { + expectSql("generate shark.TestTGF2(test, 25)", Array(25,25,25,27,29).map(_.toString).toArray ) + } + +} + +object TestTGF1 { + @Schema(spec = "values int") + def apply(test: RDD[(Int, String)], integer: Int) = { + test.map{ case Tuple2(k, v) => Tuple1(k + integer) }.filter{ case Tuple1(v) => v < 20 } + } +} + +object TestTGF2 { + def apply(sc: SharkContext, test: RDD[(Int, String)], integer: Int) = { + val rdd = test.map{ case Tuple2(k, v) => Seq(k + integer) }.filter{ case Seq(v) => v < 30 } + RDDSchema(rdd.asInstanceOf[RDD[Seq[_]]], "myvalues int") + } } From 74002658acf309c3ad98f5832f3eaf731b4cf424 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 16 Dec 2013 00:36:49 -0800 Subject: [PATCH 284/331] SAVE AS changed to AS in TGF syntax --- src/main/scala/shark/tgf/TGF.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 8b080695..311fc872 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -31,18 +31,18 @@ import shark.SharkContext * This object is responsible for handling TGF (Table Generating Function) commands. * * -- TGF Commands -- - * generate tgfname(param1, param2, ... , param_n) - * generate tgfname(param1, param2, ... , param_n) save as tablename + * GENERATE tgfname(param1, param2, ... , param_n) + * GENERATE tgfname(param1, param2, ... , param_n) AS tablename * * Parameters can either be of primitive types, e.g. int, or of type RDD[Product]. * TGF.execute() will use reflection looking for an object of name "tgfname", invoking apply() with the primitive * values. If the type of a parameter to apply() is RDD[Product], it will assume the parameter is the name of a table, * which it will turn into an RDD before invoking apply(). * - * For example, "generate MyObj(25, emp)" will invoke MyObj.apply(25, sc.sql2rdd("select * from emp")), assuming + * For example, "GENERATE MyObj(25, emp)" will invoke MyObj.apply(25, sc.sql2rdd("select * from emp")), assuming * the TGF object (MyObj) has an apply function that takes an int and an RDD[Product]. * - * The "save as" version of the command saves the output in a new table named "tablename", whereas the other version + * The "as" version of the command saves the output in a new table named "tablename", whereas the other version * returns a ResultSet * * -- Defining TGF objects -- @@ -82,7 +82,7 @@ object TGF { /** * Executes a TGF command and gives back the ResultSet. Mainly to be used from SharkContext (e.g. runSql()) - * @param sql TGF command, e.g. "generate name(params)" + * @param sql TGF command, e.g. "GENERATE name(params) AS tablename" * @param sc SharkContext * @return ResultSet containing the results of the command */ @@ -259,7 +259,7 @@ private class TGFParser extends JavaTokenParsers { * @return Tuple3 containing a table name, TGF method name and a List of parameters as strings */ def saveTgf: Parser[Tuple3[String, String, List[String]]] = { - (("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("SAVE".ci ~ "AS".ci) ~> + (("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("AS".ci) ~> ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } } From 4982435a2db20c762a8a8f3c20640034a820ef2a Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 16 Dec 2013 11:34:42 -0800 Subject: [PATCH 285/331] Java => J import prefix in HiveUtils --- src/main/scala/shark/util/HiveUtils.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index f8d595fb..806e8f92 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -17,7 +17,7 @@ package shark.util -import java.util.{Arrays => JavaArrays, ArrayList => JavaArrayList} +import java.util.{Arrays => JArrays, ArrayList => JArrayList} import java.util.{HashSet => JHashSet} import java.util.Properties @@ -67,8 +67,8 @@ private[shark] object HiveUtils { partProps: Properties, partSerDe: Deserializer): UnionStructObjectInspector = { val partCols = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - val partColNames = new JavaArrayList[String] - val partColObjectInspectors = new JavaArrayList[ObjectInspector] + val partColNames = new JArrayList[String] + val partColObjectInspectors = new JArrayList[ObjectInspector] partCols.trim().split("/").foreach { colName => partColNames.add(colName) partColObjectInspectors.add(PrimitiveObjectInspectorFactory.javaStringObjectInspector) @@ -76,7 +76,7 @@ private[shark] object HiveUtils { val partColObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( partColNames, partColObjectInspectors) - val oiList = JavaArrays.asList( + val oiList = JArrays.asList( partSerDe.getObjectInspector.asInstanceOf[StructObjectInspector], partColObjectInspector.asInstanceOf[StructObjectInspector]) // New oi is union of table + partition object inspectors @@ -98,7 +98,7 @@ private[shark] object HiveUtils { // Setup the create table descriptor with necessary information. val createTableDesc = new CreateTableDesc() createTableDesc.setTableName(tableName) - createTableDesc.setCols(new JavaArrayList[FieldSchema](schema)) + createTableDesc.setCols(new JArrayList[FieldSchema](schema)) createTableDesc.setTblProps( SharkTblProperties.initializeWithDefaults(createTableDesc.getTblProps)) createTableDesc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") From 026d15b1643a5bab079f45c69e97db54d703a609 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 16 Dec 2013 11:37:18 -0800 Subject: [PATCH 286/331] Instantiate Java HashMap for table properties initialization during direct table creation DDL. --- src/main/scala/shark/util/HiveUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index 806e8f92..dcc99911 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -18,7 +18,7 @@ package shark.util import java.util.{Arrays => JArrays, ArrayList => JArrayList} -import java.util.{HashSet => JHashSet} +import java.util.{HashMap => JHashMap, HashSet => JHashSet} import java.util.Properties import scala.collection.JavaConversions._ @@ -100,7 +100,7 @@ private[shark] object HiveUtils { createTableDesc.setTableName(tableName) createTableDesc.setCols(new JArrayList[FieldSchema](schema)) createTableDesc.setTblProps( - SharkTblProperties.initializeWithDefaults(createTableDesc.getTblProps)) + SharkTblProperties.initializeWithDefaults(new JHashMap[String, String]())) createTableDesc.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") createTableDesc.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") createTableDesc.setSerName(classOf[shark.memstore2.ColumnarSerDe].getName) From 5d111483ef24cf9beda7aa2c816c2d7d8d19c610 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 16 Dec 2013 16:24:50 -0800 Subject: [PATCH 287/331] Upgrade to scala 2.10 * Replace ClassManifests with ClassTags * Denote use of language features (implicits, existencials) * Stop using deprecated ConcurrentMap and spawn. * Stop using deprecated spark setJobDescription. * Avoid catching all Throwables in favor of catching Exception. --- project/SharkBuild.scala | 5 +- run | 4 +- .../resources/tablerdd/RDDTable_generator.py | 19 +- .../tablerdd/TableRDDGenerated_generator.py | 8 +- src/main/scala/shark/SharkConfVars.scala | 2 + src/main/scala/shark/SharkContext.scala | 129 +++---- src/main/scala/shark/SharkServer.scala | 7 +- src/main/scala/shark/api/DataTypes.java | 52 ++- src/main/scala/shark/api/JavaTableRDD.scala | 4 +- src/main/scala/shark/api/RDDTable.scala | 339 +++++++++--------- .../scala/shark/api/RDDTableFunctions.scala | 15 +- .../scala/shark/api/TableRDDGenerated.scala | 325 +++++++++-------- .../scala/shark/execution/CoGroupedRDD.scala | 2 + .../shark/execution/CommonJoinOperator.scala | 5 +- .../shark/execution/HadoopTableReader.scala | 3 +- src/main/scala/shark/execution/Operator.scala | 2 + src/main/scala/shark/execution/RDDUtils.scala | 13 +- .../scala/shark/execution/SparkLoadTask.scala | 5 +- .../scala/shark/execution/SparkTask.scala | 2 +- src/main/scala/shark/execution/package.scala | 3 +- .../memstore2/MemoryMetadataManager.scala | 6 +- .../memstore2/PartitionedMemoryTable.scala | 4 +- .../memstore2/column/ColumnIterator.scala | 3 +- .../shark/memstore2/column/ColumnType.scala | 13 +- .../column/CompressedColumnIterator.scala | 2 +- src/main/scala/shark/util/HiveUtils.scala | 11 +- 26 files changed, 511 insertions(+), 472 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 9f6e7d52..420abd77 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -30,7 +30,7 @@ object SharkBuild extends Build { val SPARK_VERSION = "0.9.0-incubating-SNAPSHOT" - val SCALA_VERSION = "2.9.3" + val SCALA_VERSION = "2.10.3" // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.1" for Apache releases, or "0.20.2-cdh3u3" for Cloudera Hadoop. @@ -65,7 +65,7 @@ object SharkBuild extends Build { organization := "edu.berkeley.cs.amplab", version := SHARK_VERSION, scalaVersion := SCALA_VERSION, - scalacOptions := Seq("-deprecation", "-unchecked", "-optimize"), + scalacOptions := Seq("-deprecation", "-unchecked", "-optimize", "-feature", "-Yinline-warnings"), parallelExecution in Test := false, // Download managed jars into lib_managed. @@ -79,6 +79,7 @@ object SharkBuild extends Build { fork := true, javaOptions += "-XX:MaxPermSize=512m", javaOptions += "-Xmx2g", + javaOptions += "-Dsun.io.serialization.extendedDebugInfo=true", testOptions in Test += Tests.Argument("-oF"), // Full stack trace on test failures diff --git a/run b/run index 48843652..eebb0637 100755 --- a/run +++ b/run @@ -1,8 +1,8 @@ #!/bin/bash # This file is used to launch Shark on the master. -export SCALA_VERSION=2.9.3 -SHARK_VERSION=0.8.0-SNAPSHOT +export SCALA_VERSION=2.10 +SHARK_VERSION=0.9.0-SNAPSHOT # Figure out where the framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/src/main/resources/tablerdd/RDDTable_generator.py b/src/main/resources/tablerdd/RDDTable_generator.py index 8b359d82..82c15371 100755 --- a/src/main/resources/tablerdd/RDDTable_generator.py +++ b/src/main/resources/tablerdd/RDDTable_generator.py @@ -46,10 +46,13 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind // *** This file is auto-generated from RDDTable_generator.py *** +import scala.language.implicitConversions + import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag object RDDTableImplicits { - private type M[T] = ClassManifest[T] + private type C[T] = ClassTag[T] """ @@ -62,7 +65,7 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind implicit def rddToTable$num[$tmlist] (rdd: RDD[($tlist)]): RDDTableFunctions = RDDTable(rdd) -""").substitute(num = x, tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) +""").substitute(num = x, tmlist = createList(1, x, "T", ": C", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) p.write(tableClass) prefix = """ @@ -70,8 +73,8 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind object RDDTable { - private type M[T] = ClassManifest[T] - private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + private type C[T] = ClassTag[T] + private def ct[T](implicit c : ClassTag[T]) = c """ p.write(prefix) @@ -82,13 +85,13 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind """ def apply[$tmlist] (rdd: RDD[($tlist)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) new RDDTableFunctions(rddSeq, Seq($mtlist)) } -""").substitute(tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4), - mtlist = createList(1, x, "m[T", "]", ", ", indent=4)) +""").substitute(tmlist = createList(1, x, "T", ": C", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4), + mtlist = createList(1, x, "ct[T", "]", ", ", indent=4)) p.write(tableClass) diff --git a/src/main/resources/tablerdd/TableRDDGenerated_generator.py b/src/main/resources/tablerdd/TableRDDGenerated_generator.py index 5744dea7..0e4626d1 100755 --- a/src/main/resources/tablerdd/TableRDDGenerated_generator.py +++ b/src/main/resources/tablerdd/TableRDDGenerated_generator.py @@ -35,6 +35,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.{TaskContext, Partition} +import scala.reflect.ClassTag + class TableSeqRDD(prev: TableRDD) extends RDD[Seq[Any]](prev) { @@ -63,16 +65,16 @@ def getSchema = prev.schema tableClass = Template( """ class TableRDD$num[$list](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple$num[$list]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == $num, "Table only has " + tableCols + " columns, expecting $num") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala index c13adbee..b0a712af 100755 --- a/src/main/scala/shark/SharkConfVars.scala +++ b/src/main/scala/shark/SharkConfVars.scala @@ -17,6 +17,8 @@ package shark +import scala.language.existentials + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index fe62122b..7ecbe37f 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -22,6 +22,7 @@ import java.util.{ArrayList => JArrayList} import scala.collection.Map import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver @@ -49,8 +50,8 @@ class SharkContext( SharkContext.init() import SharkContext._ - private type M[T] = ClassManifest[T] - private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + private type C[T] = ClassTag[T] + private def ct[T](implicit c : ClassTag[T]) = c /** @@ -122,158 +123,158 @@ class SharkContext( * a SELECT statement. */ - def sqlRdd[T1: M, T2: M](cmd: String): + def sqlRdd[T1: C, T2: C](cmd: String): RDD[Tuple2[T1, T2]] = { new TableRDD2[T1, T2](sql2rdd(cmd), - Seq(m[T1], m[T2])) + Seq(ct[T1], ct[T2])) } - def sqlRdd[T1: M, T2: M, T3: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C](cmd: String): RDD[Tuple3[T1, T2, T3]] = { new TableRDD3[T1, T2, T3](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3])) + Seq(ct[T1], ct[T2], ct[T3])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C](cmd: String): RDD[Tuple4[T1, T2, T3, T4]] = { new TableRDD4[T1, T2, T3, T4](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C](cmd: String): RDD[Tuple5[T1, T2, T3, T4, T5]] = { new TableRDD5[T1, T2, T3, T4, T5](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C](cmd: String): RDD[Tuple6[T1, T2, T3, T4, T5, T6]] = { new TableRDD6[T1, T2, T3, T4, T5, T6](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C](cmd: String): RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { new TableRDD7[T1, T2, T3, T4, T5, T6, T7](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C](cmd: String): RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { new TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C](cmd: String): RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { new TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C](cmd: String): RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { new TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C](cmd: String): RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { new TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C](cmd: String): RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { new TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C](cmd: String): RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { new TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C](cmd: String): RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { new TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C](cmd: String): RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { new TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C, T16: C](cmd: String): RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { new TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15], ct[T16])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C, T16: C, T17: C](cmd: String): RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { new TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15], ct[T16], ct[T17])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C, T16: C, T17: C, T18: C](cmd: String): RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { new TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], ct[T18])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C](cmd: String): RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19]] = { new TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], ct[T18], ct[T19])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, T20: C](cmd: String): RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20]] = { new TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], ct[T18], ct[T19], ct[T20])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, T20: C, T21: C](cmd: String): RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21]] = { new TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], ct[T18], ct[T19], ct[T20], ct[T21])) } - def sqlRdd[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, T11: M, T12: M, - T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, T20: M, T21: M, T22: M](cmd: String): + def sqlRdd[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, T11: C, T12: C, + T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, T20: C, T21: C, T22: C](cmd: String): RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22]] = { new TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22](sql2rdd(cmd), - Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], m[T11], m[T12], - m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], m[T20], m[T21], m[T22])) + Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], ct[T10], ct[T11], ct[T12], + ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], ct[T18], ct[T19], ct[T20], ct[T21], ct[T22])) } /** diff --git a/src/main/scala/shark/SharkServer.scala b/src/main/scala/shark/SharkServer.scala index b73e0919..0f42a539 100644 --- a/src/main/scala/shark/SharkServer.scala +++ b/src/main/scala/shark/SharkServer.scala @@ -28,7 +28,8 @@ import java.util.Properties import java.util.concurrent.CountDownLatch import scala.annotation.tailrec -import scala.concurrent.ops.spawn +import scala.concurrent._ +import scala.concurrent.ExecutionContext.Implicits.global import org.apache.commons.logging.LogFactory import org.apache.commons.cli.OptionBuilder @@ -93,7 +94,7 @@ object SharkServer extends LogHelper { var remoteClient = "Unknown" // Seed session ID by a random number - var sessionID = scala.Math.round(scala.Math.random * 10000000).toString + var sessionID = scala.math.round(scala.math.random * 10000000).toString var jdbcSocket: java.net.Socket = null if (t.isInstanceOf[TSocket]) { remoteClient = t.asInstanceOf[TSocket].getSocket() @@ -164,7 +165,7 @@ object SharkServer extends LogHelper { private def execLoadRdds(loadFlag: Boolean, latch:CountDownLatch) { if (!loadFlag) { latch.countDown - } else spawn { + } else future { while (!server.isServing()) {} try { val sshandler = new SharkServerHandler diff --git a/src/main/scala/shark/api/DataTypes.java b/src/main/scala/shark/api/DataTypes.java index 1f71eb05..8190e55c 100644 --- a/src/main/scala/shark/api/DataTypes.java +++ b/src/main/scala/shark/api/DataTypes.java @@ -22,8 +22,8 @@ import java.util.Map; import java.sql.Timestamp; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import org.apache.hadoop.hive.serde.Constants; @@ -107,34 +107,30 @@ public static DataType fromHiveType(String hiveType) throws UnknownDataTypeExcep } } - public static DataType fromManifest(ClassManifest m) throws UnknownDataTypeException { - if (m.equals(m(Boolean.class)) || m.equals(ClassManifest$.MODULE$.Boolean())) { - return INT; - } else if (m.equals(m(Byte.class)) || m.equals(ClassManifest$.MODULE$.Byte())) { - return TINYINT; - } else if (m.equals(m(Short.class)) || m.equals(ClassManifest$.MODULE$.Short())) { - return SMALLINT; - } else if (m.equals(m(Integer.class)) || m.equals(ClassManifest$.MODULE$.Int())) { - return INT; - } else if (m.equals(m(Long.class)) || m.equals(ClassManifest$.MODULE$.Long())) { - return BIGINT; - } else if (m.equals(m(Float.class)) || m.equals(ClassManifest$.MODULE$.Float())) { - return FLOAT; - } else if (m.equals(m(Double.class)) || m.equals(ClassManifest$.MODULE$.Double())) { - return DOUBLE; - } else if (m.equals(m(String.class))) { - return STRING; - } else if (m.equals(m(Timestamp.class))) { - return TIMESTAMP; - } else if (m.equals(m(Date.class))) { - return DATE; + public static DataType fromClassTag(ClassTag m) throws UnknownDataTypeException { + if (m.equals(ClassTag$.MODULE$.Boolean())) { + return INT; + } else if (m.equals(ClassTag$.MODULE$.Byte())){ + return TINYINT; + } else if (m.equals(ClassTag$.MODULE$.Short())) { + return SMALLINT; + } else if (m.equals(ClassTag$.MODULE$.Int())) { + return INT; + } else if (m.equals(ClassTag$.MODULE$.Long())) { + return BIGINT; + } else if (m.equals(ClassTag$.MODULE$.Float())) { + return FLOAT; + } else if (m.equals(ClassTag$.MODULE$.Double())) { + return DOUBLE; + } else if (m.equals(ClassTag$.MODULE$.apply(String.class))) { + return STRING; + } else if (m.equals(ClassTag$.MODULE$.apply(Timestamp.class))) { + return TIMESTAMP; + } else if (m.equals(ClassTag$.MODULE$.apply(Date.class))) { + return DATE; } else { - throw new UnknownDataTypeException(m.toString()); + throw new UnknownDataTypeException(m.toString()); } // TODO: binary data type. } - - private static ClassManifest m(Class cls) { - return ClassManifest$.MODULE$.fromClass(cls); - } } diff --git a/src/main/scala/shark/api/JavaTableRDD.scala b/src/main/scala/shark/api/JavaTableRDD.scala index de111173..50be2d4f 100644 --- a/src/main/scala/shark/api/JavaTableRDD.scala +++ b/src/main/scala/shark/api/JavaTableRDD.scala @@ -17,6 +17,8 @@ package shark.api +import scala.reflect.ClassTag + import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.api.java.JavaRDDLike import org.apache.spark.rdd.RDD @@ -29,7 +31,7 @@ class JavaTableRDD(val rdd: RDD[Row], val schema: Array[ColumnDesc]) override def wrapRDD(rdd: RDD[Row]): JavaTableRDD = new JavaTableRDD(rdd, schema) // Common RDD functions - override val classManifest: ClassManifest[Row] = implicitly[ClassManifest[Row]] + override val classTag: ClassTag[Row] = implicitly[ClassTag[Row]] // This shouldn't be necessary, but we seem to need this to get first() to return Row // instead of Object; possibly a compiler bug? diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index e76ea0f8..1485b2eb 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -1,3 +1,4 @@ + /* * Copyright (C) 2012 The Regents of The University California. * All rights reserved. @@ -17,116 +18,119 @@ package shark.api -// *** This file is auto-generated from rddtable_generator.py *** +// *** This file is auto-generated from RDDTable_generator.py *** + +import scala.language.implicitConversions import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag object RDDTableImplicits { - private type M[T] = ClassManifest[T] + private type C[T] = ClassTag[T] - implicit def rddToTable2[T1: M, T2: M] + implicit def rddToTable2[T1: C, T2: C] (rdd: RDD[(T1, T2)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable3[T1: M, T2: M, T3: M] + implicit def rddToTable3[T1: C, T2: C, T3: C] (rdd: RDD[(T1, T2, T3)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable4[T1: M, T2: M, T3: M, T4: M] + implicit def rddToTable4[T1: C, T2: C, T3: C, T4: C] (rdd: RDD[(T1, T2, T3, T4)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable5[T1: M, T2: M, T3: M, T4: M, T5: M] + implicit def rddToTable5[T1: C, T2: C, T3: C, T4: C, T5: C] (rdd: RDD[(T1, T2, T3, T4, T5)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable6[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M] + implicit def rddToTable6[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable7[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M] + implicit def rddToTable7[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable8[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M] + implicit def rddToTable8[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable9[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M] + implicit def rddToTable9[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable10[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M] + implicit def rddToTable10[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable11[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M] + implicit def rddToTable11[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable12[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M] + implicit def rddToTable12[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable13[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M] + implicit def rddToTable13[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable14[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M] + implicit def rddToTable14[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable15[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M] + implicit def rddToTable15[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable16[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M] + implicit def rddToTable16[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable17[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable17[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable18[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable18[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable19[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable19[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable20[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, - T20: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable20[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable21[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, - T20: M, T21: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable21[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable22[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, - T20: M, T21: M, T22: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable22[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C, T22: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22)]): RDDTableFunctions = RDDTable(rdd) @@ -134,210 +138,213 @@ object RDDTableImplicits { object RDDTable { - private type M[T] = ClassManifest[T] - private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + private type C[T] = ClassTag[T] + private def ct[T](implicit c : ClassTag[T]) = c - def apply[T1: M, T2: M] + def apply[T1: C, T2: C] (rdd: RDD[(T1, T2)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2])) } - def apply[T1: M, T2: M, T3: M] + def apply[T1: C, T2: C, T3: C] (rdd: RDD[(T1, T2, T3)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3])) } - def apply[T1: M, T2: M, T3: M, T4: M] + def apply[T1: C, T2: C, T3: C, T4: C] (rdd: RDD[(T1, T2, T3, T4)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C] (rdd: RDD[(T1, T2, T3, T4, T5)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15], m[T16])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, - T20: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], - m[T20])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19], ct[T20])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, - T20: M, T21: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], - m[T20], m[T21])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19], ct[T20], ct[T21])) } - def apply[T1: M, T2: M, T3: M, T4: M, T5: M, T6: M, T7: M, T8: M, T9: M, T10: M, - T11: M, T12: M, T13: M, T14: M, T15: M, T16: M, T17: M, T18: M, T19: M, - T20: M, T21: M, T22: M] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C, T22: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) - new RDDTableFunctions(rddSeq, Seq(m[T1], m[T2], m[T3], m[T4], m[T5], m[T6], m[T7], m[T8], m[T9], m[T10], - m[T11], m[T12], m[T13], m[T14], m[T15], m[T16], m[T17], m[T18], m[T19], - m[T20], m[T21], m[T22])) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19], ct[T20], ct[T21], ct[T22])) } } diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 50636c5e..d7378835 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -18,6 +18,7 @@ package shark.api import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag import org.apache.hadoop.hive.ql.metadata.Hive @@ -28,20 +29,20 @@ import shark.memstore2.{CacheType, TablePartitionStats, TablePartition, TablePar import shark.util.HiveUtils -class RDDTableFunctions(self: RDD[Seq[_]], manifests: Seq[ClassManifest[_]]) { +class RDDTableFunctions(self: RDD[Seq[_]], classTags: Seq[ClassTag[_]]) { def saveAsTable(tableName: String, fields: Seq[String]): Boolean = { - require(fields.size == this.manifests.size, + require(fields.size == this.classTags.size, "Number of column names != number of fields in the RDD.") - // Get a local copy of the manifests so we don't need to serialize this object. - val manifests = this.manifests + // Get a local copy of the classTags so we don't need to serialize this object. + val classTags = this.classTags val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) // Create the RDD object. val rdd = self.mapPartitionsWithIndex { case(partitionIndex, iter) => - val ois = manifests.map(HiveUtils.getJavaPrimitiveObjectInspector) + val ois = classTags.map(HiveUtils.getJavaPrimitiveObjectInspector) val builder = new TablePartitionBuilder(ois, 1000000, shouldCompress = false) for (p <- iter) { @@ -56,7 +57,7 @@ class RDDTableFunctions(self: RDD[Seq[_]], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() - var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) + var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, classTags) // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. val databaseName = Hive.get(SharkContext.hiveconf).getCurrentDatabase() @@ -69,7 +70,7 @@ class RDDTableFunctions(self: RDD[Seq[_]], manifests: Seq[ClassManifest[_]]) { // Force evaluate to put the data in memory. rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } catch { - case _ => { + case _: Exception => { // Intercept the exception thrown by SparkContext#runJob() and handle it silently. The // exception message should already be printed to the console by DDLTask#execute(). HiveUtils.dropTableInHive(tableName) diff --git a/src/main/scala/shark/api/TableRDDGenerated.scala b/src/main/scala/shark/api/TableRDDGenerated.scala index 344ee6a7..0980630c 100644 --- a/src/main/scala/shark/api/TableRDDGenerated.scala +++ b/src/main/scala/shark/api/TableRDDGenerated.scala @@ -1,3 +1,4 @@ + /* * Copyright (C) 2013 The Regents of The University California. * All rights reserved. @@ -24,6 +25,8 @@ package shark.api import org.apache.spark.rdd.RDD import org.apache.spark.{TaskContext, Partition} +import scala.reflect.ClassTag + class TableSeqRDD(prev: TableRDD) extends RDD[Seq[Any]](prev) { @@ -39,167 +42,167 @@ class TableSeqRDD(prev: TableRDD) class TableRDD1[T1](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple1[T1]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 1, "Table only has " + tableCols + " columns, expecting 1") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple1[T1]] = { prev.compute(split, context).map( row => new Tuple1[T1]( - row.getPrimitiveGeneric[T1](0) ) ) + row.getPrimitiveGeneric[T1](0) ) ) } } class TableRDD2[T1, T2](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple2[T1, T2]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 2, "Table only has " + tableCols + " columns, expecting 2") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple2[T1, T2]] = { prev.compute(split, context).map( row => new Tuple2[T1, T2]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1) ) ) } } class TableRDD3[T1, T2, T3](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple3[T1, T2, T3]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 3, "Table only has " + tableCols + " columns, expecting 3") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple3[T1, T2, T3]] = { prev.compute(split, context).map( row => new Tuple3[T1, T2, T3]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) - ) ) + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2) + ) ) } } class TableRDD4[T1, T2, T3, T4](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple4[T1, T2, T3, T4]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 4, "Table only has " + tableCols + " columns, expecting 4") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple4[T1, T2, T3, T4]] = { prev.compute(split, context).map( row => new Tuple4[T1, T2, T3, T4]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3) ) ) } } class TableRDD5[T1, T2, T3, T4, T5](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple5[T1, T2, T3, T4, T5]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 5, "Table only has " + tableCols + " columns, expecting 5") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple5[T1, T2, T3, T4, T5]] = { prev.compute(split, context).map( row => new Tuple5[T1, T2, T3, T4, T5]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4) ) ) } } class TableRDD6[T1, T2, T3, T4, T5, T6](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple6[T1, T2, T3, T4, T5, T6]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 6, "Table only has " + tableCols + " columns, expecting 6") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple6[T1, T2, T3, T4, T5, T6]] = { prev.compute(split, context).map( row => new Tuple6[T1, T2, T3, T4, T5, T6]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5) - ) ) + ) ) } } class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple7[T1, T2, T3, T4, T5, T6, T7]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 7, "Table only has " + tableCols + " columns, expecting 7") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { prev.compute(split, context).map( row => new Tuple7[T1, T2, T3, T4, T5, T6, T7]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6) ) ) @@ -207,24 +210,24 @@ class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, } class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 8, "Table only has " + tableCols + " columns, expecting 8") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { prev.compute(split, context).map( row => new Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7) ) ) @@ -232,50 +235,50 @@ class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, } class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 9, "Table only has " + tableCols + " columns, expecting 9") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { prev.compute(split, context).map( row => new Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8) - ) ) + ) ) } } class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 10, "Table only has " + tableCols + " columns, expecting 10") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { prev.compute(split, context).map( row => new Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9) ) ) @@ -284,24 +287,24 @@ class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, } class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 11, "Table only has " + tableCols + " columns, expecting 11") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { prev.compute(split, context).map( row => new Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10) ) ) @@ -310,51 +313,51 @@ class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, } class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 12, "Table only has " + tableCols + " columns, expecting 12") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { prev.compute(split, context).map( row => new Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11) - ) ) + ) ) } } class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 13, "Table only has " + tableCols + " columns, expecting 13") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { prev.compute(split, context).map( row => new Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -364,24 +367,24 @@ class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: T } class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 14, "Table only has " + tableCols + " columns, expecting 14") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { prev.compute(split, context).map( row => new Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -391,52 +394,52 @@ class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](pr } class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 15, "Table only has " + tableCols + " columns, expecting 15") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { prev.compute(split, context).map( row => new Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14) - ) ) + ) ) } } class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16](prev: TableRDD, - mans: Seq[ClassManifest[_]]) + tags: Seq[ClassTag[_]]) extends RDD[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 16, "Table only has " + tableCols + " columns, expecting 16") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { prev.compute(split, context).map( row => new Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -446,25 +449,29 @@ class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { +class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17](prev: TableRDD, + tags: Seq[ClassTag[_]]) + extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 17, "Table only has " + tableCols + " columns, expecting 17") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17]] = { prev.compute(split, context).map( row => - new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -474,58 +481,62 @@ class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { +class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18](prev: TableRDD, + tags: Seq[ClassTag[_]]) + extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 18, "Table only has " + tableCols + " columns, expecting 18") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18]] = { prev.compute(split, context).map( row => - new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17) - ) ) + ) ) } } -class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19]](prev) { +class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19](prev: TableRDD, + tags: Seq[ClassTag[_]]) + extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 19, "Table only has " + tableCols + " columns, expecting 19") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19]] = { prev.compute(split, context).map( row => - new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -536,29 +547,29 @@ T19](prev: TableRDD, } } -class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20]](prev) { +class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20](prev: TableRDD, + tags: Seq[ClassTag[_]]) + extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 20, "Table only has " + tableCols + " columns, expecting 20") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20]] = { prev.compute(split, context).map( row => - new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), @@ -569,63 +580,63 @@ T19, T20](prev: TableRDD, } } -class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20, T21](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21]](prev) { +class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21](prev: TableRDD, + tags: Seq[ClassTag[_]]) + extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 21, "Table only has " + tableCols + " columns, expecting 21") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21]] = { prev.compute(split, context).map( row => - new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), row.getPrimitiveGeneric[T13](12), row.getPrimitiveGeneric[T14](13), row.getPrimitiveGeneric[T15](14), row.getPrimitiveGeneric[T16](15), row.getPrimitiveGeneric[T17](16), row.getPrimitiveGeneric[T18](17), row.getPrimitiveGeneric[T19](18), row.getPrimitiveGeneric[T20](19), row.getPrimitiveGeneric[T21](20) - ) ) + ) ) } } -class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, -T19, T20, T21, T22](prev: TableRDD, - mans: Seq[ClassManifest[_]]) - extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22]](prev) { +class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22](prev: TableRDD, + tags: Seq[ClassTag[_]]) + extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22]](prev) { def schema = prev.schema private val tableCols = schema.size require(tableCols == 22, "Table only has " + tableCols + " columns, expecting 22") - mans.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromManifest(m) != schema(i).dataType) + tags.zipWithIndex.foreach{ case (m, i) => if (DataTypes.fromClassTag(m) != schema(i).dataType) throw new IllegalArgumentException( - "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromManifest(m) + " got " + schema(i).dataType) } + "Type mismatch on column " + (i + 1) + ", expected " + DataTypes.fromClassTag(m) + " got " + schema(i).dataType) } override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22]] = { + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22]] = { prev.compute(split, context).map( row => - new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, - T19, T20, T21, T22]( - row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), + new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + T17, T18, T19, T20, T21, T22]( + row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), row.getPrimitiveGeneric[T7](6), row.getPrimitiveGeneric[T8](7), row.getPrimitiveGeneric[T9](8), row.getPrimitiveGeneric[T10](9), row.getPrimitiveGeneric[T11](10), row.getPrimitiveGeneric[T12](11), diff --git a/src/main/scala/shark/execution/CoGroupedRDD.scala b/src/main/scala/shark/execution/CoGroupedRDD.scala index 79c5a30a..cf40b6a6 100644 --- a/src/main/scala/shark/execution/CoGroupedRDD.scala +++ b/src/main/scala/shark/execution/CoGroupedRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark +import scala.language.existentials + import java.io.{ObjectOutputStream, IOException} import java.util.{HashMap => JHashMap} diff --git a/src/main/scala/shark/execution/CommonJoinOperator.scala b/src/main/scala/shark/execution/CommonJoinOperator.scala index ff60ac3a..da258864 100755 --- a/src/main/scala/shark/execution/CommonJoinOperator.scala +++ b/src/main/scala/shark/execution/CommonJoinOperator.scala @@ -19,7 +19,8 @@ package shark.execution import java.util.{HashMap => JavaHashMap, List => JavaList, ArrayList =>JavaArrayList} -import scala.reflect.BeanProperty +import scala.beans.BeanProperty +import scala.reflect.ClassTag import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator import org.apache.hadoop.hive.ql.exec.{JoinUtil => HiveJoinUtil} @@ -102,7 +103,7 @@ abstract class CommonJoinOperator[T <: JoinDesc] extends NaryOperator[T] { } -class CartesianProduct[T >: Null : ClassManifest](val numTables: Int) { +class CartesianProduct[T >: Null : ClassTag](val numTables: Int) { val SINGLE_NULL_LIST = Seq[T](null) val EMPTY_LIST = Seq[T]() diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index 31aa84a5..1d359d64 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -149,6 +149,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf // Create local references so that the outer object isn't serialized. val tableDesc = _tableDesc val broadcastedHiveConf = _broadcastedHiveConf + val localDeserializer = partDeserializer val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) hivePartitionRDD.mapPartitions { iter => @@ -156,7 +157,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val rowWithPartArr = new Array[Object](2) // Map each tuple to a row object iter.map { value => - val deserializer = partDeserializer.newInstance() + val deserializer = localDeserializer.newInstance() deserializer.initialize(hconf, partProps) val deserializedRow = deserializer.deserialize(value) // LazyStruct rowWithPartArr.update(0, deserializedRow) diff --git a/src/main/scala/shark/execution/Operator.scala b/src/main/scala/shark/execution/Operator.scala index 2cd37ce3..a350a8a8 100755 --- a/src/main/scala/shark/execution/Operator.scala +++ b/src/main/scala/shark/execution/Operator.scala @@ -17,6 +17,8 @@ package shark.execution +import scala.language.existentials + import java.util.{List => JavaList} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ diff --git a/src/main/scala/shark/execution/RDDUtils.scala b/src/main/scala/shark/execution/RDDUtils.scala index 7d516d18..1306f9bd 100755 --- a/src/main/scala/shark/execution/RDDUtils.scala +++ b/src/main/scala/shark/execution/RDDUtils.scala @@ -18,6 +18,7 @@ package shark.execution import scala.collection.JavaConversions +import scala.reflect.ClassTag import com.google.common.collect.{Ordering => GOrdering} @@ -37,7 +38,7 @@ object RDDUtils { * Returns a UnionRDD using both RDD arguments. Any UnionRDD argument is "flattened", in that * its parent sequence of RDDs is directly passed to the UnionRDD returned. */ - def unionAndFlatten[T: ClassManifest]( + def unionAndFlatten[T: ClassTag]( rdd: RDD[T], otherRdd: RDD[T]): UnionRDD[T] = { val otherRdds: Seq[RDD[T]] = otherRdd match { @@ -72,7 +73,7 @@ object RDDUtils { * Repartition an RDD using the given partitioner. This is similar to Spark's partitionBy, * except we use the Shark shuffle serializer. */ - def repartition[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)], part: Partitioner) + def repartition[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)], part: Partitioner) : RDD[(K, V)] = { new ShuffledRDD[K, V, (K, V)](rdd, part).setSerializer(SharkEnv.shuffleSerializerName) @@ -82,7 +83,7 @@ object RDDUtils { * Sort the RDD by key. This is similar to Spark's sortByKey, except that we use * the Shark shuffle serializer. */ - def sortByKey[K <: Comparable[K]: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) + def sortByKey[K <: Comparable[K]: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) : RDD[(K, V)] = { val part = new RangePartitioner(rdd.partitions.length, rdd) @@ -97,7 +98,7 @@ object RDDUtils { /** * Return an RDD containing the top K (K smallest key) from the given RDD. */ - def topK[K <: Comparable[K]: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)], k: Int) + def topK[K <: Comparable[K]: ClassTag, V: ClassTag](rdd: RDD[(K, V)], k: Int) : RDD[(K, V)] = { // First take top K on each partition. @@ -109,7 +110,7 @@ object RDDUtils { /** * Take top K on each partition and return a new RDD. */ - def partitionTopK[K <: Comparable[K]: ClassManifest, V: ClassManifest]( + def partitionTopK[K <: Comparable[K]: ClassTag, V: ClassTag]( rdd: RDD[(K, V)], k: Int): RDD[(K, V)] = { rdd.mapPartitions(iter => topK(iter, k)) } @@ -117,7 +118,7 @@ object RDDUtils { /** * Return top K elements out of an iterator. */ - private def topK[K <: Comparable[K]: ClassManifest, V: ClassManifest]( + private def topK[K <: Comparable[K]: ClassTag, V: ClassTag]( it: Iterator[(K, V)], k: Int): Iterator[(K, V)] = { val ordering = new GOrdering[(K,V)] { override def compare(l: (K, V), r: (K, V)) = { diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index ca919b26..aaaee3ce 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -153,8 +153,9 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe val databaseName = work.databaseName val tableName = work.tableName // Set Spark's job description to be this query. - SharkEnv.sc.setJobDescription("Updating table %s.%s for a(n) %s" - .format(databaseName, tableName, work.commandType)) + SharkEnv.sc.setJobGroup( + "shark.job", + s"Updating table $databaseName.$tableName for a(n) ${work.commandType}") val hiveTable = Hive.get(conf).getTable(databaseName, tableName) // Use HadoopTableReader to help with table scans. The `conf` passed is reused across HadoopRDD // instantiations. diff --git a/src/main/scala/shark/execution/SparkTask.scala b/src/main/scala/shark/execution/SparkTask.scala index 04dea899..f878ce0c 100755 --- a/src/main/scala/shark/execution/SparkTask.scala +++ b/src/main/scala/shark/execution/SparkTask.scala @@ -89,7 +89,7 @@ class SparkTask extends HiveTask[SparkWork] with Serializable with LogHelper { terminalOp.initializeMasterOnAll() // Set Spark's job description to be this query. - SharkEnv.sc.setJobDescription(work.pctx.getContext.getCmd) + SharkEnv.sc.setJobGroup("shark.job", work.pctx.getContext.getCmd) // Set the fair scheduler's pool using mapred.fairscheduler.pool if it is defined. Option(conf.get("mapred.fairscheduler.pool")).foreach { pool => diff --git a/src/main/scala/shark/execution/package.scala b/src/main/scala/shark/execution/package.scala index 79bd903f..f8251c8a 100755 --- a/src/main/scala/shark/execution/package.scala +++ b/src/main/scala/shark/execution/package.scala @@ -17,10 +17,11 @@ package shark +import scala.language.implicitConversions + import shark.execution.serialization.KryoSerializationWrapper import shark.execution.serialization.OperatorSerializationWrapper - package object execution { type HiveDesc = java.io.Serializable // XXXDesc in Hive is the subclass of Serializable diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index adac9a01..c51ea072 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -21,7 +21,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.{HashMap=> JavaHashMap, Map => JavaMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.ConcurrentMap +import scala.collection.concurrent import org.apache.hadoop.hive.ql.metadata.Hive @@ -34,13 +34,13 @@ import shark.util.HiveUtils class MemoryMetadataManager { // Set of tables, from databaseName.tableName to Table object. - private val _tables: ConcurrentMap[String, Table] = + private val _tables: concurrent.Map[String, Table] = new ConcurrentHashMap[String, Table]() // TODO(harvey): Support stats for Hive-partitioned tables. // Set of stats, from databaseName.tableName to the stats. This is guaranteed to have the same // structure / size as the _tables map. - private val _keyToStats: ConcurrentMap[String, collection.Map[Int, TablePartitionStats]] = + private val _keyToStats: concurrent.Map[String, collection.Map[Int, TablePartitionStats]] = new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] def putStats( diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index b649d84d..80d29450 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -20,7 +20,7 @@ package shark.memstore2 import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} import scala.collection.JavaConversions._ -import scala.collection.mutable.ConcurrentMap +import scala.collection.concurrent import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -54,7 +54,7 @@ class PartitionedMemoryTable( // A map from the Hive-partition key to the RDD that contains contents of that partition. // The conventional string format for the partition key, 'col1=value1/col2=value2/...', can be // computed using MemoryMetadataManager#makeHivePartitionKeyStr() - private val _keyToPartitions: ConcurrentMap[String, RDDValue] = + private val _keyToPartitions: concurrent.Map[String, RDDValue] = new ConcurrentJavaHashMap[String, RDDValue]() // The eviction policy for this table's cached Hive-partitions. An example of how this diff --git a/src/main/scala/shark/memstore2/column/ColumnIterator.scala b/src/main/scala/shark/memstore2/column/ColumnIterator.scala index 24924162..404e456b 100644 --- a/src/main/scala/shark/memstore2/column/ColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/ColumnIterator.scala @@ -17,8 +17,9 @@ package shark.memstore2.column -import java.nio.{ByteBuffer, ByteOrder} +import scala.language.implicitConversions +import java.nio.{ByteBuffer, ByteOrder} trait ColumnIterator { diff --git a/src/main/scala/shark/memstore2/column/ColumnType.scala b/src/main/scala/shark/memstore2/column/ColumnType.scala index 7c432789..4ca62a19 100644 --- a/src/main/scala/shark/memstore2/column/ColumnType.scala +++ b/src/main/scala/shark/memstore2/column/ColumnType.scala @@ -20,6 +20,8 @@ package shark.memstore2.column import java.nio.ByteBuffer import java.sql.Timestamp +import scala.reflect.ClassTag + import org.apache.hadoop.hive.serde2.ByteStream import org.apache.hadoop.hive.serde2.`lazy`.{ByteArrayRef, LazyBinary} import org.apache.hadoop.hive.serde2.io.ByteWritable @@ -37,19 +39,18 @@ import org.apache.hadoop.io._ * @tparam T Scala data type for the column. * @tparam V Writable data type for the column. */ -sealed abstract class ColumnType[T : ClassManifest, V : ClassManifest]( +sealed abstract class ColumnType[T : ClassTag, V : ClassTag]( val typeID: Int, val defaultSize: Int) { /** - * Scala class manifest. Can be used to create primitive arrays and hash tables. + * Scala ClassTag. Can be used to create primitive arrays and hash tables. */ - def scalaManifest: ClassManifest[T] = classManifest[T] + def scalaTag = implicitly[ClassTag[T]] /** - * Scala class manifest for the writable type. Can be used to create primitive arrays and - * hash tables. + * Scala ClassTag. Can be used to create primitive arrays and hash tables. */ - def writableManifest: ClassManifest[V] = classManifest[V] + def writableScalaTag = implicitly[ClassTag[V]] /** * Extract a value out of the buffer at the buffer's current position. diff --git a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala index 3cb5f441..5d74a61c 100644 --- a/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala +++ b/src/main/scala/shark/memstore2/column/CompressedColumnIterator.scala @@ -108,7 +108,7 @@ class DictDecoder[V](buffer: ByteBuffer, columnType: ColumnType[_, V]) extends I // decompressed value. private val _dictionary: Array[V] = { val size = buffer.getInt() - val arr = columnType.writableManifest.newArray(size) + val arr = columnType.writableScalaTag.newArray(size) var count = 0 while (count < size) { val writable = columnType.newWritable() diff --git a/src/main/scala/shark/util/HiveUtils.scala b/src/main/scala/shark/util/HiveUtils.scala index f8d595fb..8b7d58db 100644 --- a/src/main/scala/shark/util/HiveUtils.scala +++ b/src/main/scala/shark/util/HiveUtils.scala @@ -21,6 +21,7 @@ import java.util.{Arrays => JavaArrays, ArrayList => JavaArrayList} import java.util.{HashSet => JHashSet} import java.util.Properties +import scala.reflect.ClassTag import scala.collection.JavaConversions._ import org.apache.hadoop.hive.conf.HiveConf @@ -43,8 +44,8 @@ import shark.memstore2.SharkTblProperties private[shark] object HiveUtils { - def getJavaPrimitiveObjectInspector(m: ClassManifest[_]): PrimitiveObjectInspector = { - getJavaPrimitiveObjectInspector(DataTypes.fromManifest(m)) + def getJavaPrimitiveObjectInspector(c: ClassTag[_]): PrimitiveObjectInspector = { + getJavaPrimitiveObjectInspector(DataTypes.fromClassTag(c)) } def getJavaPrimitiveObjectInspector(t: DataType): PrimitiveObjectInspector = t match { @@ -89,10 +90,10 @@ private[shark] object HiveUtils { def createTableInHive( tableName: String, columnNames: Seq[String], - columnTypes: Seq[ClassManifest[_]], + columnTypes: Seq[ClassTag[_]], hiveConf: HiveConf = new HiveConf): Boolean = { - val schema = columnNames.zip(columnTypes).map { case (colName, manifest) => - new FieldSchema(colName, DataTypes.fromManifest(manifest).hiveName, "") + val schema = columnNames.zip(columnTypes).map { case (colName, classTag) => + new FieldSchema(colName, DataTypes.fromClassTag(classTag).hiveName, "") } // Setup the create table descriptor with necessary information. From 94d54e4a583b7430ba293143fc2d34a1ebd218b2 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Tue, 17 Dec 2013 00:45:16 -0800 Subject: [PATCH 288/331] Bug in saveAsTable() [Thanks to @harveyfeng] --- src/main/scala/shark/api/RDDTableFunctions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index 50636c5e..d6ffa49e 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -56,7 +56,7 @@ class RDDTableFunctions(self: RDD[Seq[_]], manifests: Seq[ClassManifest[_]]) { Iterator(builder.build()) }.persist() - var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests) + var isSucessfulCreateTable = HiveUtils.createTableInHive(tableName, fields, manifests, Hive.get().getConf()) // Put the table in the metastore. Only proceed if the DDL statement is executed successfully. val databaseName = Hive.get(SharkContext.hiveconf).getCurrentDatabase() From f36cac85662bb6d710f846f924aa9e2447dac5eb Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Tue, 17 Dec 2013 00:45:33 -0800 Subject: [PATCH 289/331] More TGF test cases --- src/test/scala/shark/SQLSuite.scala | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 5fb3428f..be6dde4c 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} import shark.tgf.{RDDSchema, Schema} +import scala.util.Try class SQLSuite extends FunSuite with BeforeAndAfterAll { @@ -1155,13 +1156,26 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { ////////////////////////////////////////////////////////////////////////////// test("Simple TGFs") { - expectSql("generate shark.TestTGF1(test, 15)", Array(15,15,15,17,19).map(_.toString).toArray ) + expectSql("generate shark.TestTGF1(test, 15)", Array(15,15,15,17,19).map(_.toString).toArray) } - test("Advanced TGFs with SharkContext and dynamic schemas") { - expectSql("generate shark.TestTGF2(test, 25)", Array(25,25,25,27,29).map(_.toString).toArray ) + test("Saving simple TGFs") { + sc.sql("drop table if exists TGFTestTable") + sc.runSql("generate shark.TestTGF1(test, 15) as TGFTestTable") + expectSql("select * from TGFTestTable", Array(15,15,15,17,19).map(_.toString).toArray) + sc.sql("drop table if exists TGFTestTable") } + test("Advanced TGFs") { + expectSql("generate shark.TestTGF2(test, 25)", Array(25,25,25,27,29).map(_.toString).toArray) + } + + test("Saving advanced TGFs") { + sc.sql("drop table if exists TGFTestTable2") + sc.runSql("generate shark.TestTGF2(test, 25) as TGFTestTable2") + expectSql("select * from TGFTestTable2", Array(25,25,25,27,29).map(_.toString).toArray) + sc.sql("drop table if exists TGFTestTable2") + } } object TestTGF1 { From 1ffe230152884b660a90270a4cf4880fcd23b3b1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 17 Dec 2013 16:11:39 -0800 Subject: [PATCH 290/331] Avoid reloading RDD for remote CLI mode. --- src/main/scala/shark/SharkCliDriver.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index f7e524d8..32565865 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -53,7 +53,7 @@ object SharkCliDriver { private var prompt = "shark" private var prompt2 = " " // when ';' is not yet seen. private var transport:TSocket = _ - + installSignalHandler() /** @@ -214,7 +214,7 @@ object SharkCliDriver { clientTransportTSocketField.setAccessible(true) transport = clientTransportTSocketField.get(ss).asInstanceOf[TSocket] - + var ret = 0 var prefix = "" @@ -265,10 +265,9 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe // because the Hive unit tests do not go through the main() code path. if (!ss.isRemoteMode()) { SharkEnv.init() - } - - if (reloadRdds) { - TableRecovery.reloadRdds(processCmd(_)) + if (reloadRdds) { + TableRecovery.reloadRdds(processCmd(_)) + } } def this() = this(false) From 17c440e40dbc12b965a7ad6588921f84b5c3933a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 17 Dec 2013 10:59:49 -0800 Subject: [PATCH 291/331] remove deprecated use of spawn --- src/test/scala/shark/SharkServerSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/test/scala/shark/SharkServerSuite.scala b/src/test/scala/shark/SharkServerSuite.scala index e5df4f98..cb53d6bd 100644 --- a/src/test/scala/shark/SharkServerSuite.scala +++ b/src/test/scala/shark/SharkServerSuite.scala @@ -10,7 +10,8 @@ import scala.collection.JavaConversions._ import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.matchers.ShouldMatchers -import scala.concurrent.ops._ +import scala.concurrent._ +import ExecutionContext.Implicits.global /** * Test for the Shark server. @@ -57,7 +58,7 @@ class SharkServerSuite extends FunSuite with BeforeAndAfterAll with ShouldMatche // Spawn a thread to read the output from the forked process. // Note that this is necessary since in some configurations, log4j could be blocked // if its output to stderr are not read, and eventually blocking the entire test suite. - spawn { + future { while (true) { val stdout = readFrom(inputReader) val stderr = readFrom(errorReader) From 4f98b0093819fb084feaee91aabf3f22abbad46b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 17 Dec 2013 16:12:46 -0800 Subject: [PATCH 292/331] add sleep before talking to shark server --- src/test/scala/shark/SharkServerSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/scala/shark/SharkServerSuite.scala b/src/test/scala/shark/SharkServerSuite.scala index cb53d6bd..1310ca04 100644 --- a/src/test/scala/shark/SharkServerSuite.scala +++ b/src/test/scala/shark/SharkServerSuite.scala @@ -79,6 +79,7 @@ class SharkServerSuite extends FunSuite with BeforeAndAfterAll with ShouldMatche } test("test query execution against a shark server") { + Thread.sleep(5*1000) // I know... Gross. However, without this the tests fail non-deterministically. val dataFilePath = TestUtils.dataFilePath + "/kv1.txt" val stmt = createStatement() From 062b4b4a3f4744e7465249fdf7424ff68f2c0289 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Thu, 2 Jan 2014 16:42:38 -0800 Subject: [PATCH 293/331] Add -nc to wget to avoid it "not clobber" (or download a *.1 version of it) the file if it already exists. --- bin/dev/run-tests-from-scratch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/dev/run-tests-from-scratch b/bin/dev/run-tests-from-scratch index a19a5898..c8ec8e7c 100755 --- a/bin/dev/run-tests-from-scratch +++ b/bin/dev/run-tests-from-scratch @@ -226,7 +226,7 @@ fi # Download Scala if SCALA_HOME is not specified. #################################################################### if [ "x$SCALA_HOME" == "x" ] ; then - wget $SCALA_DOWNLOAD_PATH + wget -nc $SCALA_DOWNLOAD_PATH tar xvfz scala*tgz export SCALA_HOME="$WORKSPACE/scala-$SCALA_VERSION" fi From 29137aee5834c1944bf84f0b66ed9fa731b5068a Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Thu, 2 Jan 2014 17:57:21 -0800 Subject: [PATCH 294/331] remove old downloads of scala*tgz before downlaoding and untar-ing a new version. --- bin/dev/run-tests-from-scratch | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bin/dev/run-tests-from-scratch b/bin/dev/run-tests-from-scratch index c8ec8e7c..06703ccb 100755 --- a/bin/dev/run-tests-from-scratch +++ b/bin/dev/run-tests-from-scratch @@ -226,7 +226,8 @@ fi # Download Scala if SCALA_HOME is not specified. #################################################################### if [ "x$SCALA_HOME" == "x" ] ; then - wget -nc $SCALA_DOWNLOAD_PATH + rm -rf ./scala*tgz + wget $SCALA_DOWNLOAD_PATH tar xvfz scala*tgz export SCALA_HOME="$WORKSPACE/scala-$SCALA_VERSION" fi From e2e7305a3475c1e278fb2a99d3eb3aaae8d08538 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 3 Jan 2014 15:02:22 -0800 Subject: [PATCH 295/331] Update to pass conf object in various places. --- src/main/scala/shark/execution/CoGroupedRDD.scala | 2 +- src/main/scala/shark/execution/LateralViewJoinOperator.scala | 3 ++- .../scala/shark/execution/serialization/JavaSerializer.scala | 3 ++- .../scala/shark/execution/serialization/KryoSerializer.scala | 3 ++- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/execution/CoGroupedRDD.scala b/src/main/scala/shark/execution/CoGroupedRDD.scala index cf40b6a6..e5806e34 100644 --- a/src/main/scala/shark/execution/CoGroupedRDD.scala +++ b/src/main/scala/shark/execution/CoGroupedRDD.scala @@ -117,7 +117,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) } values } - val serializer = SparkEnv.get.serializerManager.get(SharkEnv.shuffleSerializerName) + val serializer = SparkEnv.get.serializerManager.get(SharkEnv.shuffleSerializerName, SparkEnv.get.conf) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, itsSplitIndex, itsSplit) => { // Read them from the parent diff --git a/src/main/scala/shark/execution/LateralViewJoinOperator.scala b/src/main/scala/shark/execution/LateralViewJoinOperator.scala index b2d05953..603e3b87 100755 --- a/src/main/scala/shark/execution/LateralViewJoinOperator.scala +++ b/src/main/scala/shark/execution/LateralViewJoinOperator.scala @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.plan.{LateralViewJoinDesc, SelectDesc} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector} +import org.apache.spark.SparkEnv import org.apache.spark.rdd.RDD import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} @@ -151,7 +152,7 @@ class LateralViewJoinOperator extends NaryOperator[LateralViewJoinDesc] { */ object KryoSerializerToString { - @transient val kryoSer = new SparkKryoSerializer + @transient val kryoSer = new SparkKryoSerializer(SparkEnv.get.conf) def serialize[T](o: T): String = { val bytes = kryoSer.newInstance().serialize(o).array() diff --git a/src/main/scala/shark/execution/serialization/JavaSerializer.scala b/src/main/scala/shark/execution/serialization/JavaSerializer.scala index a98cb95c..df6ab31d 100644 --- a/src/main/scala/shark/execution/serialization/JavaSerializer.scala +++ b/src/main/scala/shark/execution/serialization/JavaSerializer.scala @@ -19,11 +19,12 @@ package shark.execution.serialization import java.nio.ByteBuffer +import org.apache.spark.SparkEnv import org.apache.spark.serializer.{JavaSerializer => SparkJavaSerializer} object JavaSerializer { - @transient val ser = new SparkJavaSerializer + @transient val ser = new SparkJavaSerializer(SparkEnv.get.conf) def serialize[T](o: T): Array[Byte] = { ser.newInstance().serialize(o).array() diff --git a/src/main/scala/shark/execution/serialization/KryoSerializer.scala b/src/main/scala/shark/execution/serialization/KryoSerializer.scala index c4764979..3b6ba082 100644 --- a/src/main/scala/shark/execution/serialization/KryoSerializer.scala +++ b/src/main/scala/shark/execution/serialization/KryoSerializer.scala @@ -19,6 +19,7 @@ package shark.execution.serialization import java.nio.ByteBuffer +import org.apache.spark.SparkEnv import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} @@ -29,7 +30,7 @@ import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} */ object KryoSerializer { - @transient val ser = new SparkKryoSerializer + @transient val ser = new SparkKryoSerializer(SparkEnv.get.conf) def serialize[T](o: T): Array[Byte] = { ser.newInstance().serialize(o).array() From 5018dd442f97952f0b650034a3d5b8354d2fa189 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 3 Jan 2014 15:41:45 -0800 Subject: [PATCH 296/331] Also update test failing shark test cases to use a conf object for testing that skips loading external settings. --- .../execution/serialization/SerializationSuite.scala | 5 +++-- src/test/scala/shark/memstore2/TablePartitionSuite.scala | 9 +++++---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/test/scala/shark/execution/serialization/SerializationSuite.scala b/src/test/scala/shark/execution/serialization/SerializationSuite.scala index 90383a2a..2f1001de 100755 --- a/src/test/scala/shark/execution/serialization/SerializationSuite.scala +++ b/src/test/scala/shark/execution/serialization/SerializationSuite.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.scalatest.FunSuite +import org.apache.spark.SparkConf import org.apache.spark.serializer.{JavaSerializer => SparkJavaSerializer} @@ -51,7 +52,7 @@ class SerializationSuite extends FunSuite { val ois = KryoSerializationWrapper(new ArrayBuffer[ObjectInspector]) ois.value += oi - val ser = new SparkJavaSerializer + val ser = new SparkJavaSerializer() val bytes = ser.newInstance().serialize(ois) val desered = ser.newInstance() .deserialize[KryoSerializationWrapper[ArrayBuffer[ObjectInspector]]](bytes) @@ -67,7 +68,7 @@ class SerializationSuite extends FunSuite { operator.localHiveOp = new org.apache.hadoop.hive.ql.exec.FileSinkOperator val opWrapped = OperatorSerializationWrapper(operator) - val ser = new SparkJavaSerializer + val ser = new SparkJavaSerializer(new SparkConf(false)) val bytes = ser.newInstance().serialize(opWrapped) val desered = ser.newInstance() .deserialize[OperatorSerializationWrapper[SharkFileSinkOperator]](bytes) diff --git a/src/test/scala/shark/memstore2/TablePartitionSuite.scala b/src/test/scala/shark/memstore2/TablePartitionSuite.scala index 047d4071..843cb1b1 100644 --- a/src/test/scala/shark/memstore2/TablePartitionSuite.scala +++ b/src/test/scala/shark/memstore2/TablePartitionSuite.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite +import org.apache.spark.SparkConf import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -31,7 +32,7 @@ class TablePartitionSuite extends FunSuite { val col2 = Array[Byte](1, 2, 3) val tp = new TablePartition(3, Array(ByteBuffer.wrap(col1), ByteBuffer.wrap(col2))) - val ser = new JavaSerializer + val ser = new JavaSerializer(new SparkConf(false)) val bytes = ser.newInstance().serialize(tp) val tp1 = ser.newInstance().deserialize[TablePartition](bytes) assert(tp1.numRows === 3) @@ -58,7 +59,7 @@ class TablePartitionSuite extends FunSuite { col2.rewind() val tp = new TablePartition(3, Array(col1, col2)) - val ser = new JavaSerializer + val ser = new JavaSerializer(new SparkConf(false)) val bytes = ser.newInstance().serialize(tp) val tp1 = ser.newInstance().deserialize[TablePartition](bytes) assert(tp1.numRows === 3) @@ -77,7 +78,7 @@ class TablePartitionSuite extends FunSuite { val col2 = Array[Byte](1, 2, 3) val tp = new TablePartition(3, Array(ByteBuffer.wrap(col1), ByteBuffer.wrap(col2))) - val ser = new KryoSerializer + val ser = new KryoSerializer(new SparkConf(false)) val bytes = ser.newInstance().serialize(tp) val tp1 = ser.newInstance().deserialize[TablePartition](bytes) assert(tp1.numRows === 3) @@ -104,7 +105,7 @@ class TablePartitionSuite extends FunSuite { col2.rewind() val tp = new TablePartition(3, Array(col1, col2)) - val ser = new KryoSerializer + val ser = new KryoSerializer(new SparkConf(false)) val bytes = ser.newInstance().serialize(tp) val tp1 = ser.newInstance().deserialize[TablePartition](bytes) assert(tp1.numRows === 3) From 7e6f6e13026e91ea652fd374440f04af8a5053f3 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 3 Jan 2014 15:55:14 -0800 Subject: [PATCH 297/331] One more failing test case that needs conf object passed. --- .../shark/execution/serialization/SerializationSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/shark/execution/serialization/SerializationSuite.scala b/src/test/scala/shark/execution/serialization/SerializationSuite.scala index 2f1001de..2f25698d 100755 --- a/src/test/scala/shark/execution/serialization/SerializationSuite.scala +++ b/src/test/scala/shark/execution/serialization/SerializationSuite.scala @@ -52,7 +52,7 @@ class SerializationSuite extends FunSuite { val ois = KryoSerializationWrapper(new ArrayBuffer[ObjectInspector]) ois.value += oi - val ser = new SparkJavaSerializer() + val ser = new SparkJavaSerializer(new SparkConf(false)) val bytes = ser.newInstance().serialize(ois) val desered = ser.newInstance() .deserialize[KryoSerializationWrapper[ArrayBuffer[ObjectInspector]]](bytes) From 391faec60d85800a0cdaa6c416a7fc29255c4e24 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 5 Jan 2014 15:11:34 -0800 Subject: [PATCH 298/331] Updated ShuffleSerializer to accept a SparkConf as constructor argument. --- .../shark/execution/serialization/ShuffleSerializer.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala index ae91086b..a9091e46 100644 --- a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala +++ b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import org.apache.hadoop.io.BytesWritable +import org.apache.spark.SparkConf import org.apache.spark.serializer.DeserializationStream import org.apache.spark.serializer.{SerializationStream, Serializer, SerializerInstance} @@ -48,7 +49,7 @@ import shark.execution.{ReduceKey, ReduceKeyReduceSide} * into a hash table. We want to reduce the size of the hash table. Having the BytesWritable wrapper * would increase the size of the hash table by another 16 bytes per key-value pair. */ -class ShuffleSerializer extends Serializer { +class ShuffleSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance } From e9a8fac514a54e13bf387709a843bfc374bc38c5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 5 Jan 2014 15:56:58 -0800 Subject: [PATCH 299/331] Fix ShuffleSerializerSuite compilation. --- .../shark/execution/serialization/ShuffleSerializer.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala index a9091e46..e4eba584 100644 --- a/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala +++ b/src/main/scala/shark/execution/serialization/ShuffleSerializer.scala @@ -50,6 +50,10 @@ import shark.execution.{ReduceKey, ReduceKeyReduceSide} * would increase the size of the hash table by another 16 bytes per key-value pair. */ class ShuffleSerializer(conf: SparkConf) extends Serializer { + + // A no-arg constructor since conf is not needed in this serializer. + def this() = this(null) + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance } From f7aa9d3073d34d972c7f68e7bdca98ec597a863e Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Sun, 5 Jan 2014 16:27:55 -0800 Subject: [PATCH 300/331] Update build proces for spark now that sbt is no longer shipped with Spark. --- bin/dev/run-tests-from-scratch | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/bin/dev/run-tests-from-scratch b/bin/dev/run-tests-from-scratch index 06703ccb..085e38bb 100755 --- a/bin/dev/run-tests-from-scratch +++ b/bin/dev/run-tests-from-scratch @@ -12,6 +12,7 @@ # Set up config vars using env vars or defaults; parse cmd line flags. ##################################################################### SHARK_PROJ_DIR_DEFAULT="$(cd `dirname $0`/../../; pwd)" +SBT_OPTS_DEFAULT="-Xms512M -Xmx2048M -Xss1M -XX:+CMSClassUnloadingEnabled -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=256m -XX:+UseCodeCacheFlushing" SPARK_MEM_DEFAULT=4g SHARK_MASTER_MEM_DEFAULT=4g SPARK_KV_JAVA_OPTS_DEFAULT=("-Dspark.local.dir=/tmp " "-Dspark.kryoserializer.buffer.mb=10 ") @@ -49,6 +50,10 @@ else fi fi +if [ "x$SBT_OPTS" == "x" ] ; then + SBT_OPTS=$SBT_OPTS_DEFAULT +fi + if [ "x$SPARK_MEM" == "x" ] ; then export SPARK_MEM=$SPARK_MEM_DEFAULT fi @@ -117,6 +122,7 @@ Required Options: Optional configuration environment variables: SHARK_PROJ_DIR (default: "$SHARK_PROJ_DIR_DEFAULT") SCALA_HOME (default: Scala version ${SCALA_VERSION} will be downloaded and used) + SBT_OPTS (default: "$SBT_OPTS_DEFAULT") SPARK_MEM (default: $SPARK_MEM_DEFAULT) SHARK_MASTER_MEM (default: $SHARK_MASTER_MEM_DEFAULT) SPARK_JAVA_OPTS (default: "${SPARK_KV_JAVA_OPTS_DEFAULT[@]}") @@ -252,7 +258,8 @@ else export SPARK_HADOOP_VERSION=$SPARK_HADOOP_VERSION export SPARK_WITH_YARN=$SPARK_WITH_YARN # Build spark and push the jars to local Ivy/Maven caches. - sbt/sbt clean publish-local + wget -nc http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/0.13.0/sbt-launch.jar + java $SBT_OPTS -jar sbt-launch.jar clean publish-local popd fi export SPARK_HOME="$WORKSPACE/spark" From 103daffa702bebee07ec9f2a30afb1ab6794041f Mon Sep 17 00:00:00 2001 From: Harvey Date: Wed, 8 Jan 2014 18:42:55 -0800 Subject: [PATCH 301/331] Fix SerializationSuite failures due to uninitialized SharkEnv and SharkContext. Conflicts: src/main/scala/shark/SharkEnv.scala src/main/scala/shark/execution/serialization/KryoSerializer.scala --- src/main/scala/shark/SharkContext.scala | 2 -- src/main/scala/shark/SharkEnv.scala | 29 +++++++++---------- .../serialization/KryoSerializer.scala | 13 +++++++-- .../serialization/SerializationSuite.scala | 7 +++-- 4 files changed, 30 insertions(+), 21 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 7ecbe37f..7076be13 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -358,5 +358,3 @@ object SharkContext { // A dummy init to make sure the object is properly initialized. def init() {} } - - diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 7d1745ca..ee634399 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -24,28 +24,27 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.StatsReportListener import shark.api.JavaSharkContext -import shark.execution.serialization.ShuffleSerializer -import shark.memstore2.MemoryMetadataManager +import shark.execution.serialization.{KryoSerializer, ShuffleSerializer} +import shark.memstore2.{MemoryMetadataManager, Table} import shark.tachyon.TachyonUtilImpl /** A singleton object for the master program. The slaves should not access this. */ object SharkEnv extends LogHelper { - def init(): SparkContext = { + def init(): SharkContext = { if (sc == null) { - sc = new SparkContext( - if (System.getenv("MASTER") == null) "local" else System.getenv("MASTER"), - "Shark::" + java.net.InetAddress.getLocalHost.getHostName, - System.getenv("SPARK_HOME"), - Nil, - executorEnvVars) - sc.addSparkListener(new StatsReportListener()) + val jobName = "Shark::" + java.net.InetAddress.getLocalHost.getHostName + val master = System.getenv("MASTER") + initWithSharkContext(jobName, master) } + KryoSerializer.initWithSharkContext(sc) sc } - def initWithSharkContext(jobName: String, master: String = System.getenv("MASTER")) + def initWithSharkContext( + jobName: String = "Shark::" + java.net.InetAddress.getLocalHost.getHostName, + master: String = System.getenv("MASTER")) : SharkContext = { if (sc != null) { sc.stop() @@ -58,16 +57,16 @@ object SharkEnv extends LogHelper { Nil, executorEnvVars) sc.addSparkListener(new StatsReportListener()) - sc.asInstanceOf[SharkContext] + KryoSerializer.initWithSharkContext(sc) + sc } def initWithSharkContext(newSc: SharkContext): SharkContext = { if (sc != null) { sc.stop() } - sc = newSc - sc.asInstanceOf[SharkContext] + sc } def initWithJavaSharkContext(jobName: String): JavaSharkContext = { @@ -96,7 +95,7 @@ object SharkEnv extends LogHelper { val activeSessions = new HashSet[String] - var sc: SparkContext = _ + var sc: SharkContext = _ val shuffleSerializerName = classOf[ShuffleSerializer].getName diff --git a/src/main/scala/shark/execution/serialization/KryoSerializer.scala b/src/main/scala/shark/execution/serialization/KryoSerializer.scala index 3b6ba082..a5c39148 100644 --- a/src/main/scala/shark/execution/serialization/KryoSerializer.scala +++ b/src/main/scala/shark/execution/serialization/KryoSerializer.scala @@ -19,9 +19,10 @@ package shark.execution.serialization import java.nio.ByteBuffer -import org.apache.spark.SparkEnv +import org.apache.spark.SparkConf import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} +import shark.SharkContext /** * Java object serialization using Kryo. This is much more efficient, but Kryo @@ -30,7 +31,15 @@ import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} */ object KryoSerializer { - @transient val ser = new SparkKryoSerializer(SparkEnv.get.conf) + @transient var ser: SparkKryoSerializer = _ + + def initWithSharkContext(sc: SharkContext) { + ser = new SparkKryoSerializer(sc.sparkEnv.conf) + } + + def initWithSparkConf(sparkConf: SparkConf) { + ser = new SparkKryoSerializer(sparkConf) + } def serialize[T](o: T): Array[Byte] = { ser.newInstance().serialize(o).array() diff --git a/src/test/scala/shark/execution/serialization/SerializationSuite.scala b/src/test/scala/shark/execution/serialization/SerializationSuite.scala index 2f25698d..cc2845c2 100755 --- a/src/test/scala/shark/execution/serialization/SerializationSuite.scala +++ b/src/test/scala/shark/execution/serialization/SerializationSuite.scala @@ -46,13 +46,16 @@ object SerializationSuite { class SerializationSuite extends FunSuite { + // Initialize the Shark KryoSerializer singleton. + KryoSerializer.initWithSparkConf(new SparkConf(loadDefaults = false)) + test("Java serializing object inspectors") { val oi = PrimitiveObjectInspectorFactory.javaStringObjectInspector val ois = KryoSerializationWrapper(new ArrayBuffer[ObjectInspector]) ois.value += oi - val ser = new SparkJavaSerializer(new SparkConf(false)) + val ser = new SparkJavaSerializer(new SparkConf(loadDefaults = false)) val bytes = ser.newInstance().serialize(ois) val desered = ser.newInstance() .deserialize[KryoSerializationWrapper[ArrayBuffer[ObjectInspector]]](bytes) @@ -68,7 +71,7 @@ class SerializationSuite extends FunSuite { operator.localHiveOp = new org.apache.hadoop.hive.ql.exec.FileSinkOperator val opWrapped = OperatorSerializationWrapper(operator) - val ser = new SparkJavaSerializer(new SparkConf(false)) + val ser = new SparkJavaSerializer(new SparkConf(loadDefaults = false)) val bytes = ser.newInstance().serialize(opWrapped) val desered = ser.newInstance() .deserialize[OperatorSerializationWrapper[SharkFileSinkOperator]](bytes) From c663dd5fdd71b5cc2767fa2117e09acd28f28929 Mon Sep 17 00:00:00 2001 From: Harvey Date: Wed, 8 Jan 2014 18:43:25 -0800 Subject: [PATCH 302/331] Always launch shark-shell with Java --- bin/ext/sharkserver.sh | 4 ---- run | 22 ++++++++-------------- 2 files changed, 8 insertions(+), 18 deletions(-) diff --git a/bin/ext/sharkserver.sh b/bin/ext/sharkserver.sh index e93aadee..de4c08a8 100644 --- a/bin/ext/sharkserver.sh +++ b/bin/ext/sharkserver.sh @@ -18,10 +18,6 @@ THISSERVICE=sharkserver export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} " -# Use Java to launch Shark otherwise the unit tests cannot properly kill -# the server process. -export SHARK_LAUNCH_WITH_JAVA=1 - sharkserver() { echo "Starting the Shark Server" exec $FWDIR/run shark.SharkServer "$@" diff --git a/run b/run index eebb0637..f24c201d 100755 --- a/run +++ b/run @@ -138,22 +138,16 @@ export JAVA_OPTS export ANT_OPTS=$JAVA_OPTS if [ "x$RUNNER" == "x" ] ; then - if [ "$SHARK_LAUNCH_WITH_JAVA" == "1" ]; then - CLASSPATH+=":$SCALA_HOME/lib/scala-library.jar" - CLASSPATH+=":$SCALA_HOME/lib/scala-compiler.jar" - CLASSPATH+=":$SCALA_HOME/lib/jline.jar" - if [ -n "$JAVA_HOME" ]; then - RUNNER="${JAVA_HOME}/bin/java" - else - RUNNER=java - fi - # The JVM doesn't read JAVA_OPTS by default so we need to pass it in - EXTRA_ARGS="$JAVA_OPTS" + CLASSPATH+=":$SCALA_HOME/lib/scala-library.jar" + CLASSPATH+=":$SCALA_HOME/lib/scala-compiler.jar" + CLASSPATH+=":$SCALA_HOME/lib/jline.jar" + if [ -n "$JAVA_HOME" ]; then + RUNNER="${JAVA_HOME}/bin/java" else - SCALA=${SCALA_HOME}/bin/scala - RUNNER="$SCALA -cp \"$CLASSPATH\"" - EXTRA_ARGS="" + RUNNER=java fi + # The JVM doesn't read JAVA_OPTS by default so we need to pass it in + EXTRA_ARGS="$JAVA_OPTS" fi exec $RUNNER $EXTRA_ARGS "$@" From 40903504a77be5436d834e6cf0cc56d429a0dc0d Mon Sep 17 00:00:00 2001 From: Harvey Date: Thu, 9 Jan 2014 14:29:16 -0800 Subject: [PATCH 303/331] Support stats for cached partitioned tables, take 1 --- .../scala/shark/api/RDDTableFunctions.scala | 5 +- .../execution/MemoryStoreSinkOperator.scala | 91 ++++++------------- .../scala/shark/execution/SharkDDLTask.scala | 4 +- .../scala/shark/execution/SparkLoadTask.scala | 41 ++------- .../scala/shark/execution/TableReader.scala | 87 ++++++++++++------ .../shark/execution/TableScanOperator.scala | 20 ++-- .../memstore2/MemoryMetadataManager.scala | 28 +----- .../scala/shark/memstore2/MemoryTable.scala | 45 ++++++++- .../memstore2/PartitionedMemoryTable.scala | 66 +++++++++----- src/main/scala/shark/memstore2/Table.scala | 37 +++++++- src/test/scala/shark/SQLSuite.scala | 4 +- 11 files changed, 229 insertions(+), 199 deletions(-) diff --git a/src/main/scala/shark/api/RDDTableFunctions.scala b/src/main/scala/shark/api/RDDTableFunctions.scala index d7378835..6697cdb4 100644 --- a/src/main/scala/shark/api/RDDTableFunctions.scala +++ b/src/main/scala/shark/api/RDDTableFunctions.scala @@ -65,7 +65,6 @@ class RDDTableFunctions(self: RDD[Seq[_]], classTags: Seq[ClassTag[_]]) { // Create an entry in the MemoryMetadataManager. val newTable = SharkEnv.memoryMetadataManager.createMemoryTable( databaseName, tableName, CacheType.MEMORY) - newTable.tableRDD = rdd try { // Force evaluate to put the data in memory. rdd.context.runJob(rdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) @@ -79,9 +78,7 @@ class RDDTableFunctions(self: RDD[Seq[_]], classTags: Seq[ClassTag[_]]) { isSucessfulCreateTable = false } } - - // Gather the partition statistics. - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, statsAcc.value.toMap) + newTable.put(rdd, statsAcc.value.toMap) } return isSucessfulCreateTable } diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index cc23c73b..41716e6b 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -143,40 +143,25 @@ class MemoryStoreSinkOperator extends TerminalOperator { outputRDD.context.runJob( outputRDD, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } else { - // Put the table in Spark block manager. - op.logInfo("Putting %sRDD for %s.%s in Spark block manager".format( - if (useUnionRDD) "Union" else "", databaseName, tableName)) - - outputRDD.persist(StorageLevel.MEMORY_AND_DISK) - - val queryOutputRDD = outputRDD - if (useUnionRDD) { - // Handle an INSERT INTO command. - val previousRDDOpt: Option[RDD[TablePartition]] = if (isHivePartitioned) { - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - databaseName, tableName).get - partitionedTable.getPartition(hivePartitionKey) - } else { - SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName).map(_.tableRDD) - } - outputRDD = previousRDDOpt match { - case Some(previousRDD) => { - // If the RDD for a table or Hive-partition has already been created, then take a union - // of the current data and the SELECT output. - hasPreviousRDDForUnion = true - RDDUtils.unionAndFlatten(queryOutputRDD, previousRDD) - } - // This is an INSERT into a new Hive-partition. - case None => queryOutputRDD - } - } // Run a job on the RDD that contains the query output to force the data into the memory // store. The statistics will also be collected by 'statsAcc' during job execution. - queryOutputRDD.context.runJob( - queryOutputRDD, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + if (cacheMode == CacheType.MEMORY) { + outputRDD.persist(StorageLevel.MEMORY_AND_DISK) + } else if (cacheMode == CacheType.MEMORY_ONLY { + outputRDD.persist(StorageLevel.MEMORY_ONLY) + } + outputRDD.context.runJob( + outputRDD, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) } - if (isHivePartitioned) { + // Put the table in Spark block manager or Tachyon. + op.logInfo("Putting %sRDD for %s.%s in %s store".format( + if (useUnionRDD) "Union" else "", + databaseName, + tableName, + if (cacheMode == CacheType.NONE) "disk" else cacheMode.toString)) + + val tableStats = if (isHivePartitioned) { val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( databaseName, tableName).get outputRDD.setName("%s.%s(%s)".format(databaseName, tableName, hivePartitionKey)) @@ -184,50 +169,34 @@ class MemoryStoreSinkOperator extends TerminalOperator { // An RDD for the Hive partition already exists, so update its metadata entry in // 'partitionedTable'. assert(outputRDD.isInstanceOf[UnionRDD[_]]) - partitionedTable.updatePartition(hivePartitionKey, outputRDD) + partitionedTable.updatePartition(hivePartitionKey, outputRDD, statsAcc.value) } else { // This is a new Hive-partition. Add a new metadata entry in 'partitionedTable'. - partitionedTable.putPartition(hivePartitionKey, outputRDD) + partitionedTable.putPartition(hivePartitionKey, outputRDD, statsAcc.value.toMap) } + // Stats should be updated at this point. + partitionedTable.getStats(hivePartitionKey).get } else { outputRDD.setName(tableName) // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( databaseName, tableName, cacheMode)) - memoryTable.tableRDD = outputRDD - } - - // TODO(harvey): Get this to work for Hive-partitioned tables. It should be a simple - // 'tableName' + 'hivePartitionKey' concatentation. Though whether stats should belong in - // memstore2.Table should be considered... - val columnStats = if (useUnionRDD && hasPreviousRDDForUnion) { - // Combine stats for the two RDDs that were combined into UnionRDD. - val numPartitions = statsAcc.value.toMap.size - val currentStats = statsAcc.value - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) match { - case Some(otherIndexToStats) => { - for ((otherIndex, tableStats) <- otherIndexToStats) { - currentStats.append((otherIndex + numPartitions, tableStats)) - } - } - case _ => Unit + if (useUnionRDD) { + memoryTable.update(outputRDD, statsAcc.value) + } else { + memoryTable.put(outputRDD, statsAcc.value.toMap) } - currentStats.toMap - } else { - statsAcc.value.toMap - } - - // Get the column statistics back to the cache manager. - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, columnStats) - - if (tachyonWriter != null) { - tachyonWriter.updateMetadata(ByteBuffer.wrap(JavaSerializer.serialize(columnStats))) + // TODO: For Tachyon support, have Shark Tables manage stats store and updates. + if (tachyonWriter != null) { + tachyonWriter.updateMetadata(ByteBuffer.wrap(JavaSerializer.serialize(statsAcc.value.toMap))) + } + memoryTable.getStats.get } if (SharkConfVars.getBoolVar(localHconf, SharkConfVars.MAP_PRUNING_PRINT_DEBUG)) { - columnStats.foreach { case(index, tableStats) => - println("Partition " + index + " " + tableStats.toString) + tableStats.foreach { case(index, tablePartitionStats) => + println("Partition " + index + " " + tablePartitionStats.toString) } } diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index df3e9157..ddbb8373 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -102,10 +102,8 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val memoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( dbName, tableName, cacheMode) // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. - memoryTable.tableRDD = new EmptyRDD(SharkEnv.sc) + memoryTable.put(new EmptyRDD(SharkEnv.sc)) } - // Add an empty stats entry to the Shark metastore. - SharkEnv.memoryMetadataManager.putStats(dbName, tableName, Map[Int, TablePartitionStats]()) } /** diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index aaaee3ce..019d391a 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -267,20 +267,13 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe tableSchema, hadoopReader.broadcastedHiveConf, serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) - memoryTable.tableRDD = work.commandType match { + work.commandType match { case (SparkLoadWork.CommandTypes.OVERWRITE | SparkLoadWork.CommandTypes.NEW_ENTRY) => - tablePartitionRDD + memoryTable.put(tablePartitionRDD, tableStats.toMap) case SparkLoadWork.CommandTypes.INSERT => { - // Union the previous and new RDDs, and their respective table stats. - val unionedRDD = RDDUtils.unionAndFlatten(tablePartitionRDD, memoryTable.tableRDD) - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName ) match { - case Some(previousStatsMap) => SparkLoadTask.unionStatsMaps(tableStats, previousStatsMap) - case None => Unit - } - unionedRDD + memoryTable.update(tablePartitionRDD, tableStats) } } - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } /** @@ -361,21 +354,12 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe hadoopReader.broadcastedHiveConf, unionOI) // Determine how to cache the table RDD created. - val tableOpt = partitionedTable.getPartition(partitionKey) - if (tableOpt.isDefined && (work.commandType == SparkLoadWork.CommandTypes.INSERT)) { - val previousRDD = tableOpt.get - partitionedTable.updatePartition(partitionKey, - RDDUtils.unionAndFlatten(tablePartitionRDD, previousRDD)) - // Union stats for the previous RDD with the new RDD loaded. - val previousStatsMapOpt = SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) - assert(SharkEnv.memoryMetadataManager.getStats(databaseName, tableName).isDefined, - "Stats for %s.%s should be defined for an INSERT operation, but are missing.". - format(databaseName, tableName)) - SparkLoadTask.unionStatsMaps(tableStats, previousStatsMapOpt.get) + if (partitionedTable.containsPartition(partitionKey) && + (work.commandType == SparkLoadWork.CommandTypes.INSERT)) { + partitionedTable.updatePartition(partitionKey, tablePartitionRDD, tableStats) } else { - partitionedTable.putPartition(partitionKey, tablePartitionRDD) + partitionedTable.putPartition(partitionKey, tablePartitionRDD, tableStats.toMap) } - SharkEnv.memoryMetadataManager.putStats(databaseName, tableName, tableStats.toMap) } } @@ -419,15 +403,4 @@ object SparkLoadTask { } serDeProps } - - private def unionStatsMaps( - targetStatsMap: ArrayBuffer[(Int, TablePartitionStats)], - otherStatsMap: Iterable[(Int, TablePartitionStats)] - ): ArrayBuffer[(Int, TablePartitionStats)] = { - val targetStatsMapSize = targetStatsMap.size - for ((otherIndex, tableStats) <- otherStatsMap) { - targetStatsMap.append((otherIndex + targetStatsMapSize, tableStats)) - } - targetStatsMap - } } diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 40cce0d9..56ea5b23 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -17,6 +17,8 @@ package shark.execution +import scala.collection.mutable.{ArrayBuffer, HashMap} + import org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_PARTITION_COLUMNS import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} @@ -27,7 +29,7 @@ import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} import shark.{LogHelper, SharkEnv} import shark.api.QueryExecutionException import shark.execution.serialization.JavaSerializer -import shark.memstore2.{MemoryMetadataManager, TablePartition, TablePartitionStats} +import shark.memstore2.{MemoryMetadataManager, Table, TablePartition, TablePartitionStats} import shark.tachyon.TachyonException @@ -36,7 +38,7 @@ import shark.tachyon.TachyonException * type of table storage: HeapTableReader for Shark tables in Spark's block manager, * TachyonTableReader for tables in Tachyon, and HadoopTableReader for Hive tables in a filesystem. */ -trait TableReader extends LogHelper{ +trait TableReader extends LogHelper { def makeRDDForTable(hiveTable: HiveTable): RDD[_] @@ -44,8 +46,20 @@ trait TableReader extends LogHelper{ } +trait MemoryTableReader extends TableReader { + + def makeRDDForTableWithStats( + hiveTable: HiveTable + ): (RDD[_], collection.Map[Int, TablePartitionStats]) + + def makeRDDForPartitionedTableWithStats( + partitions: Seq[HivePartition] + ): (RDD[_], collection.Map[Int, TablePartitionStats]) + +} + /** Helper class for scanning tables stored in Tachyon. */ -class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { +class TachyonTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReader { // Split from 'databaseName.tableName' private val _tableNameSplit = _tableDesc.getTableName.split('.') @@ -53,35 +67,38 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { private val _tableName = _tableNameSplit(1) override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { + makeRDDForTableWithStats(hiveTable)._1 + } + + override def makeRDDForTableWithStats( + hiveTable: HiveTable + ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { // Table is in Tachyon. val tableKey = SharkEnv.makeTachyonTableKey(_databaseName, _tableName) if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") } - logInfo("Loading table " + tableKey + " from Tachyon.") - - // True if stats for the target table is missing from the Shark metastore, and should be fetched - // and deserialized from Tachyon's metastore. This can happen if that table was created in a - // previous Shark session, since Shark's metastore is not persistent. - val shouldFetchStatsFromTachyon = SharkEnv.memoryMetadataManager.getStats( - _databaseName, _tableName).isEmpty - if (shouldFetchStatsFromTachyon) { - val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) - val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( - statsByteBuffer.array()) - logInfo("Loading table " + tableKey + " stats from Tachyon.") - SharkEnv.memoryMetadataManager.putStats(_databaseName, _tableName, indexToStats) - } - SharkEnv.tachyonUtil.createRDD(tableKey) + val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) + val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( + statsByteBuffer.array()) + logInfo("Fetching table " + tableKey + " stats from Tachyon.") + (SharkEnv.tachyonUtil.createRDD(tableKey), indexToStats) } override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") } + + override def makeRDDForPartitionedTableWithStats( + partitions: Seq[HivePartition] + ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { + throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") + } + } /** Helper class for scanning tables stored in Spark's block manager */ -class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { +class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReader { // Split from 'databaseName.tableName' private val _tableNameSplit = _tableDesc.getTableName.split('.') @@ -89,7 +106,9 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { private val _tableName = _tableNameSplit(1) /** Fetches the RDD for `_tableName` from the Shark metastore. */ - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { + override def makeRDDForTableWithStats( + hiveTable: HiveTable + ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { logInfo("Loading table %s.%s from Spark block manager".format(_databaseName, _tableName)) val tableOpt = SharkEnv.memoryMetadataManager.getMemoryTable(_databaseName, _tableName) if (tableOpt.isEmpty) { @@ -97,7 +116,11 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { } val table = tableOpt.get - table.tableRDD + (table.getRDD.get, table.getStats.get) + } + + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { + makeRDDForTableWithStats(hiveTable)._1 } /** @@ -107,7 +130,10 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. */ - override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { + override def makeRDDForPartitionedTableWithStats( + partitions: Seq[HivePartition] + ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { + val unionedStats = new ArrayBuffer[(Int, TablePartitionStats)] val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) // Get partition field info @@ -132,10 +158,11 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { } val hivePartitionedTable = hivePartitionedTableOpt.get - val hivePartitionRDDOpt = hivePartitionedTable.getPartition(partitionKeyStr) - if (hivePartitionRDDOpt.isEmpty) throwMissingPartitionException(partitionKeyStr) - val hivePartitionRDD = hivePartitionRDDOpt.get - + val rddAndStatsOpt = hivePartitionedTable.getPartitionAndStats(partitionKeyStr) + if (rddAndStatsOpt.isEmpty) throwMissingPartitionException(partitionKeyStr) + val (hivePartitionRDD, hivePartitionStats) = (rddAndStatsOpt.get._1, rddAndStatsOpt.get._2) + // Union any stats previously collected. + Table.mergeStats(unionedStats, hivePartitionStats) hivePartitionRDD.mapPartitions { iter => if (iter.hasNext) { // Map each tuple to a row object @@ -152,12 +179,16 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { } } if (hivePartitionRDDs.size > 0) { - new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs) + (new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs), unionedStats.toMap) } else { - new EmptyRDD[Object](SharkEnv.sc) + (new EmptyRDD[Object](SharkEnv.sc), new HashMap[Int, TablePartitionStats]) } } + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { + makeRDDForPartitionedTableWithStats(partitions)._1 + } + /** * Thrown if the table identified by the (_databaseName, _tableName) pair cannot be found in * the Shark metastore. diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 18d20c39..5be02981 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -124,16 +124,16 @@ class TableScanOperator extends TopOperator[TableScanDesc] { if (table.isPartitioned) { return tableReader.makeRDDForPartitionedTable(parts) } else { - val tableRdd = tableReader.makeRDDForTable(table) - return createPrunedRdd(databaseName, tableName, tableRdd) + val (tableRdd, stats) = tableReader.makeRDDForTableWithStats(table) + return createPrunedRdd(databaseName, tableName, tableRdd, stats) } } else if (cacheMode == CacheType.TACHYON) { val tableReader = new TachyonTableReader(tableDesc) if (table.isPartitioned) { return tableReader.makeRDDForPartitionedTable(parts) } else { - val tableRdd = tableReader.makeRDDForTable(table) - return createPrunedRdd(databaseName, tableName, tableRdd) + val (tableRdd, stats) = tableReader.makeRDDForTableWithStats(table) + return createPrunedRdd(databaseName, tableName, tableRdd, stats) } } else { // Table is a Hive table on HDFS (or other Hadoop storage). @@ -141,13 +141,11 @@ class TableScanOperator extends TopOperator[TableScanDesc] { } } - private def createPrunedRdd(databaseName: String, tableName: String, rdd: RDD[_]): RDD[_] = { - // Stats used for map pruning. - val indexToStatsOpt: Option[collection.Map[Int, TablePartitionStats]] = - SharkEnv.memoryMetadataManager.getStats(databaseName, tableName) - assert(indexToStatsOpt.isDefined, "Stats not found for table " + tableName) - val indexToStats = indexToStatsOpt.get - + private def createPrunedRdd( + databaseName: String, + tableName: String, + rdd: RDD[_], + indexToStats: collection.Map[Int, TablePartitionStats]): RDD[_] = { // Run map pruning if the flag is set, there exists a filter predicate on // the input table and we have statistics on the table. val columnsUsed = new ColumnPruner(this, table).columnsUsed diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index c51ea072..74f483ad 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -37,27 +37,6 @@ class MemoryMetadataManager { private val _tables: concurrent.Map[String, Table] = new ConcurrentHashMap[String, Table]() - // TODO(harvey): Support stats for Hive-partitioned tables. - // Set of stats, from databaseName.tableName to the stats. This is guaranteed to have the same - // structure / size as the _tables map. - private val _keyToStats: concurrent.Map[String, collection.Map[Int, TablePartitionStats]] = - new ConcurrentHashMap[String, collection.Map[Int, TablePartitionStats]] - - def putStats( - databaseName: String, - tableName: String, - stats: collection.Map[Int, TablePartitionStats]) { - val tableKey = makeTableKey(databaseName, tableName) - _keyToStats.put(tableKey, stats) - } - - def getStats( - databaseName: String, - tableName: String): Option[collection.Map[Int, TablePartitionStats]] = { - val tableKey = makeTableKey(databaseName, tableName) - _keyToStats.get(tableKey) - } - def isHivePartitioned(databaseName: String, tableName: String): Boolean = { val tableKey = makeTableKey(databaseName, tableName) _tables.get(tableKey) match { @@ -130,18 +109,16 @@ class MemoryMetadataManager { val oldTableKey = makeTableKey(databaseName, oldName) val newTableKey = makeTableKey(databaseName, newName) - val statsValueEntry = _keyToStats.remove(oldTableKey).get val tableValueEntry = _tables.remove(oldTableKey).get tableValueEntry.tableName = newTableKey - _keyToStats.put(newTableKey, statsValueEntry) _tables.put(newTableKey, tableValueEntry) } } /** * Used to drop a table from the Spark in-memory cache and/or disk. All metadata tracked by Shark - * (e.g. entry in '_keyToStats' if the table isn't Hive-partitioned) is deleted as well. + * is deleted as well. * * @return Option::isEmpty() is true of there is no MemoryTable (and RDD) corresponding to 'key' * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will @@ -151,7 +128,6 @@ class MemoryMetadataManager { val tableKey = makeTableKey(databaseName, tableName) // Remove MemoryTable's entry from Shark metadata. - _keyToStats.remove(tableKey) val tableValue: Option[Table] = _tables.remove(tableKey) tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) } @@ -213,7 +189,7 @@ object MemoryMetadataManager { unpersistedRDD = Some(unionedRDD) } } else { - unpersistedRDD = Some(RDDUtils.unpersistRDD(table.asInstanceOf[MemoryTable].tableRDD)) + unpersistedRDD = Some(RDDUtils.unpersistRDD(table.asInstanceOf[MemoryTable].getRDD.get)) } unpersistedRDD } diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index e82e2fa1..bfd5163d 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -19,6 +19,11 @@ package shark.memstore2 import org.apache.spark.rdd.RDD +import scala.collection.mutable.{Buffer, HashMap} + +import shark.execution.RDDUtils + + /** * A metadata container for a table in Shark that's backed by an RDD. */ @@ -28,10 +33,42 @@ private[shark] class MemoryTable( cacheMode: CacheType.CacheType) extends Table(databaseName, tableName, cacheMode) { - // RDD that contains the contents of this table. - private var _tableRDD: RDD[TablePartition] = _ + var rddValueOpt: Option[RDDValue] = None + + def put( + newRDD: RDD[TablePartition], + newStats: collection.Map[Int, TablePartitionStats] = new HashMap[Int, TablePartitionStats]() + ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { + val prevRDDAndStatsOpt = rddValueOpt.map(_.toTuple) + if (rddValueOpt.isDefined) { + rddValueOpt.foreach { rddValue => + rddValue.rdd = newRDD + rddValue.stats = newStats + } + } else { + rddValueOpt = Some(new RDDValue(newRDD, newStats)) + } + prevRDDAndStatsOpt + } + + def update( + newRDD: RDD[TablePartition], + newStats: Buffer[(Int, TablePartitionStats)] + ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { + val prevRDDAndStatsOpt = rddValueOpt.map(_.toTuple) + if (rddValueOpt.isDefined) { + val (prevRDD, prevStats) = (prevRDDAndStatsOpt.get._1, prevRDDAndStatsOpt.get._2) + val updatedRDDValue = rddValueOpt.get + updatedRDDValue.rdd = RDDUtils.unionAndFlatten(prevRDD, newRDD) + updatedRDDValue.stats = Table.mergeStats(newStats, prevStats).toMap + } else { + put(newRDD, newStats.toMap) + } + prevRDDAndStatsOpt + } + + def getRDD = rddValueOpt.map(_.rdd) - def tableRDD: RDD[TablePartition] = _tableRDD + def getStats = rddValueOpt.map(_.stats) - def tableRDD_= (rdd: RDD[TablePartition]) = _tableRDD = rdd } diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 80d29450..820bf5f0 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -21,6 +21,7 @@ import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} import scala.collection.JavaConversions._ import scala.collection.concurrent +import scala.collection.mutable.{Buffer, HashMap} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -42,18 +43,9 @@ class PartitionedMemoryTable( cacheMode: CacheType.CacheType) extends Table(databaseName, tableName, cacheMode) { - /** - * A simple, mutable wrapper for an RDD. This is needed so that a entry maintained by a - * CachePolicy's underlying data structure, such as the LinkedHashMap for LRUCachePolicy, can be - * updated without causing an eviction. - * The value entires for a single key in - * `_keyToPartitions` and `_cachePolicy` will reference the same RDDValue object. - */ - class RDDValue(var rdd: RDD[TablePartition]) - // A map from the Hive-partition key to the RDD that contains contents of that partition. // The conventional string format for the partition key, 'col1=value1/col2=value2/...', can be - // computed using MemoryMetadataManager#makeHivePartitionKeyStr() + // computed using MemoryMetadataManager#makeHivePartitionKeyStr(). private val _keyToPartitions: concurrent.Map[String, RDDValue] = new ConcurrentJavaHashMap[String, RDDValue]() @@ -61,50 +53,74 @@ class PartitionedMemoryTable( // can be set from the CLI: // `TBLPROPERTIES("shark.partition.cachePolicy", "LRUCachePolicy")`. // If 'None', then all partitions will be put in memory. + // + // Since RDDValue is mutable, entries maintained by a CachePolicy's underlying data structure, + // such as the LinkedHashMap for LRUCachePolicy, can be updated without causing an eviction. + // The value entires for a single key in + // `_keyToPartitions` and `_cachePolicy` will reference the same RDDValue object. private var _cachePolicy: CachePolicy[String, RDDValue] = _ def containsPartition(partitionKey: String): Boolean = _keyToPartitions.contains(partitionKey) def getPartition(partitionKey: String): Option[RDD[TablePartition]] = { + getPartitionAndStats(partitionKey).map(_._1) + } + + def getStats(partitionKey: String): Option[collection.Map[Int, TablePartitionStats]] = { + getPartitionAndStats(partitionKey).map(_._2) + } + + def getPartitionAndStats( + partitionKey: String + ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { val rddValueOpt: Option[RDDValue] = _keyToPartitions.get(partitionKey) if (rddValueOpt.isDefined) _cachePolicy.notifyGet(partitionKey) - rddValueOpt.map(_.rdd) + rddValueOpt.map(_.toTuple) } def putPartition( partitionKey: String, newRDD: RDD[TablePartition], - isUpdate: Boolean = false): Option[RDD[TablePartition]] = { + newStats: collection.Map[Int, TablePartitionStats] = new HashMap[Int, TablePartitionStats]() + ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { val rddValueOpt = _keyToPartitions.get(partitionKey) - val prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) - val newRDDValue = new RDDValue(newRDD) + val prevRDDAndStats = rddValueOpt.map(_.toTuple) + val newRDDValue = new RDDValue(newRDD, newStats) _keyToPartitions.put(partitionKey, newRDDValue) _cachePolicy.notifyPut(partitionKey, newRDDValue) - prevRDD + prevRDDAndStats } def updatePartition( partitionKey: String, - updatedRDD: RDD[TablePartition]): Option[RDD[TablePartition]] = { - val rddValueOpt = _keyToPartitions.get(partitionKey) - val prevRDD: Option[RDD[TablePartition]] = rddValueOpt.map(_.rdd) - if (rddValueOpt.isDefined) { + newRDD: RDD[TablePartition], + newStats: Buffer[(Int, TablePartitionStats)] + ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { + val prevRDDAndStatsOpt = getPartitionAndStats(partitionKey) + if (prevRDDAndStatsOpt.isDefined) { + val (prevRDD, prevStats) = (prevRDDAndStatsOpt.get._1, prevRDDAndStatsOpt.get._2) // This is an update of an old value, so update the RDDValue's `rdd` entry. // Don't notify the `_cachePolicy`. Assumes that getPartition() has already been called to // obtain the value of the previous RDD. - // An RDD update refers to the RDD created from a transform or union. - val updatedRDDValue = rddValueOpt.get - updatedRDDValue.rdd = updatedRDD + // An RDD update refers to the RDD created from an INSERT. + val updatedRDDValue = _keyToPartitions.get(partitionKey).get + updatedRDDValue.rdd = RDDUtils.unionAndFlatten(prevRDD, newRDD) + updatedRDDValue.stats = Table.mergeStats(newStats, prevStats).toMap + } else { + // No previous RDDValue entry currently exists for `partitionKey`, so add one. + putPartition(partitionKey, newRDD, newStats.toMap) } - prevRDD + prevRDDAndStatsOpt } - def removePartition(partitionKey: String): Option[RDD[TablePartition]] = { + def removePartition( + partitionKey: String + ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { val rddRemoved = _keyToPartitions.remove(partitionKey) if (rddRemoved.isDefined) { _cachePolicy.notifyRemove(partitionKey) } - rddRemoved.map(_.rdd) + rddRemoved.map(_.toTuple) } def setPartitionCachePolicy(cachePolicyStr: String, fallbackMaxSize: Int) { diff --git a/src/main/scala/shark/memstore2/Table.scala b/src/main/scala/shark/memstore2/Table.scala index 8cb573f2..ae7f451f 100644 --- a/src/main/scala/shark/memstore2/Table.scala +++ b/src/main/scala/shark/memstore2/Table.scala @@ -17,6 +17,13 @@ package shark.memstore2 +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.rdd.RDD + +import scala.collection.mutable.Buffer + + /** * A container for table metadata managed by Shark and Spark. Subclasses are responsible for * how RDDs are set, stored, and accessed. @@ -28,4 +35,32 @@ package shark.memstore2 private[shark] abstract class Table( var databaseName: String, var tableName: String, - var cacheMode: CacheType.CacheType) + var cacheMode: CacheType.CacheType) { + + /** + * A mutable wrapper for an RDD and stats for its partitions. + */ + class RDDValue( + var rdd: RDD[TablePartition], + var stats: collection.Map[Int, TablePartitionStats]) { + + def toTuple = (rdd, stats) + } +} + +object Table { + + /** + * Merges contents of `otherStatsMaps` into `targetStatsMap`. + */ + def mergeStats( + targetStatsMap: Buffer[(Int, TablePartitionStats)], + otherStatsMap: Iterable[(Int, TablePartitionStats)] + ): Buffer[(Int, TablePartitionStats)] = { + val targetStatsMapSize = targetStatsMap.size + for ((otherIndex, tableStats) <- otherStatsMap) { + targetStatsMap.append((otherIndex + targetStatsMapSize, tableStats)) + } + targetStatsMap + } +} diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index ed190cfe..90033b5c 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -728,13 +728,13 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { sc.sql("insert into table flat_cached select * from test") val tableName = "flat_cached" var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(DEFAULT_DB_NAME, tableName).get - var unionRDD = memoryTable.tableRDD.asInstanceOf[UnionRDD[_]] + var unionRDD = memoryTable.getRDD.get.asInstanceOf[UnionRDD[_]] val numParentRDDs = unionRDD.rdds.size assert(isFlattenedUnionRDD(unionRDD)) // Insert another set of query results. The flattening should kick in here. sc.sql("insert into table flat_cached select * from test") - unionRDD = memoryTable.tableRDD.asInstanceOf[UnionRDD[_]] + unionRDD = memoryTable.getRDD.get.asInstanceOf[UnionRDD[_]] assert(isFlattenedUnionRDD(unionRDD)) assert(unionRDD.rdds.size == numParentRDDs + 1) } From dbd51ed0ab6fbd6ba821386a290a23e621a803d5 Mon Sep 17 00:00:00 2001 From: Harvey Date: Thu, 9 Jan 2014 17:31:10 -0800 Subject: [PATCH 304/331] Decouple SharkContext from SQLSuite. Put it in a SharkRunner singleton so that we can run SQL queries across different test suites. --- .../execution/MemoryStoreSinkOperator.scala | 2 +- src/test/scala/shark/SQLSuite.scala | 101 ++------------- src/test/scala/shark/SharkRunner.scala | 120 ++++++++++++++++++ src/test/scala/shark/SortSuite.scala | 37 +++--- 4 files changed, 147 insertions(+), 113 deletions(-) create mode 100644 src/test/scala/shark/SharkRunner.scala diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index 41716e6b..bf04af6c 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -147,7 +147,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // store. The statistics will also be collected by 'statsAcc' during job execution. if (cacheMode == CacheType.MEMORY) { outputRDD.persist(StorageLevel.MEMORY_AND_DISK) - } else if (cacheMode == CacheType.MEMORY_ONLY { + } else if (cacheMode == CacheType.MEMORY_ONLY) { outputRDD.persist(StorageLevel.MEMORY_ONLY) } outputRDD.context.runJob( diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 90033b5c..aff0f392 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -21,7 +21,6 @@ import java.util.{HashMap => JavaHashMap} import scala.collection.JavaConversions._ -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME @@ -33,99 +32,19 @@ import shark.api.QueryExecutionException import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} -class SQLSuite extends FunSuite with BeforeAndAfterAll { +class SQLSuite extends FunSuite { - val WAREHOUSE_PATH = TestUtils.getWarehousePath() - val METASTORE_PATH = TestUtils.getMetastorePath() - val MASTER = "local" val DEFAULT_DB_NAME = DEFAULT_DATABASE_NAME val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" - var sc: SharkContext = _ - var sharkMetastore: MemoryMetadataManager = _ + var sc: SharkContext = SharkRunner.init() + var sharkMetastore: MemoryMetadataManager = SharkEnv.memoryMetadataManager - /** - * Tables accessible by any test in SQLSuite. Their properties should remain constant across - * tests. - */ - def loadTables() { - // test - sc.runSql("drop table if exists test") - sc.runSql("CREATE TABLE test (key INT, val STRING)") - sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test") - sc.runSql("drop table if exists test_cached") - sc.runSql("CREATE TABLE test_cached AS SELECT * FROM test") + // Shortcut methods to cut down on line length + def expectSql(sql: String, expectedResults: Array[String], sort: Boolean = true) = + SharkRunner.expectSql(sql, expectedResults, sort) - // test_null - sc.runSql("drop table if exists test_null") - sc.runSql("CREATE TABLE test_null (key INT, val STRING)") - sc.runSql("""LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' - INTO TABLE test_null""") - sc.runSql("drop table if exists test_null_cached") - sc.runSql("CREATE TABLE test_null_cached AS SELECT * FROM test_null") - - // clicks - sc.runSql("drop table if exists clicks") - sc.runSql("""create table clicks (id int, click int) - row format delimited fields terminated by '\t'""") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/clicks.txt' - OVERWRITE INTO TABLE clicks""") - sc.runSql("drop table if exists clicks_cached") - sc.runSql("create table clicks_cached as select * from clicks") - - // users - sc.runSql("drop table if exists users") - sc.runSql("""create table users (id int, name string) - row format delimited fields terminated by '\t'""") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/users.txt' - OVERWRITE INTO TABLE users""") - sc.runSql("drop table if exists users_cached") - sc.runSql("create table users_cached as select * from users") - - // test1 - sc.sql("drop table if exists test1") - sc.sql("""CREATE TABLE test1 (id INT, test1val ARRAY) - row format delimited fields terminated by '\t'""") - sc.sql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/test1.txt' INTO TABLE test1") - sc.sql("drop table if exists test1_cached") - sc.sql("CREATE TABLE test1_cached AS SELECT * FROM test1") - } - - override def beforeAll() { - sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) - - sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + - METASTORE_PATH + ";create=true") - sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) - - sharkMetastore = SharkEnv.memoryMetadataManager - - // second db - sc.sql("create database if not exists seconddb") - - loadTables() - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - private def expectSql(sql: String, expectedResults: Array[String], sort: Boolean = true) { - val sharkResults: Array[String] = sc.runSql(sql).results.map(_.mkString("\t")).toArray - val results = if (sort) sharkResults.sortWith(_ < _) else sharkResults - val expected = if (sort) expectedResults.sortWith(_ < _) else expectedResults - assert(results.corresponds(expected)(_.equals(_)), - "In SQL: " + sql + "\n" + - "Expected: " + expected.mkString("\n") + "; got " + results.mkString("\n")) - } - - // A shortcut for single row results. - private def expectSql(sql: String, expectedResult: String) { - expectSql(sql, Array(expectedResult)) - } + def expectSql(sql: String, expectedResults: String) = SharkRunner.expectSql(sql, expectedResults) private def createCachedPartitionedTable( tableName: String, @@ -153,7 +72,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( DEFAULT_DB_NAME, tableName).get - return partitionedTable + partitionedTable } def isFlattenedUnionRDD(unionRDD: UnionRDD[_]) = { @@ -166,7 +85,7 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { partSpecOpt: Option[Map[String, String]] = None) { // Check that the table is in memory and is a unified view. val sharkTableOpt = sharkMetastore.getTable(DEFAULT_DB_NAME, cachedTableName) - assert(sharkTableOpt.isDefined, "Table %s cannot be found in the Shark meatstore") + assert(sharkTableOpt.isDefined, "Table %s cannot be found in the Shark metastore") assert(sharkTableOpt.get.cacheMode == CacheType.MEMORY, "'shark.cache' field for table %s is not CacheType.MEMORY") @@ -1146,6 +1065,6 @@ class SQLSuite extends FunSuite with BeforeAndAfterAll { (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) } // Finally, reload all tables. - loadTables() + SharkRunner.loadTables() } } diff --git a/src/test/scala/shark/SharkRunner.scala b/src/test/scala/shark/SharkRunner.scala new file mode 100644 index 00000000..5ea8047d --- /dev/null +++ b/src/test/scala/shark/SharkRunner.scala @@ -0,0 +1,120 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME + +import shark.memstore2.MemoryMetadataManager + + +object SharkRunner { + + val WAREHOUSE_PATH = TestUtils.getWarehousePath() + val METASTORE_PATH = TestUtils.getMetastorePath() + val MASTER = "local" + + var sc: SharkContext = _ + var sharkMetastore: MemoryMetadataManager = _ + + + def init(): SharkContext = { + if (sc == null) { + sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) + + sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + + METASTORE_PATH + ";create=true") + sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) + + sharkMetastore = SharkEnv.memoryMetadataManager + + // second db + sc.sql("create database if not exists seconddb") + + loadTables() + } + sc + } + + /** + * Tables accessible by any test. Their properties should remain constant across + * tests. + */ + def loadTables() { + require(sc != null, "call init() to instantiate a SharkContext first") + + // Use the default namespace + sc.runSql("USE " + DEFAULT_DATABASE_NAME) + + // test + sc.runSql("drop table if exists test") + sc.runSql("CREATE TABLE test (key INT, val STRING)") + sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test") + sc.runSql("drop table if exists test_cached") + sc.runSql("CREATE TABLE test_cached AS SELECT * FROM test") + + // test_null + sc.runSql("drop table if exists test_null") + sc.runSql("CREATE TABLE test_null (key INT, val STRING)") + sc.runSql("""LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' + INTO TABLE test_null""") + sc.runSql("drop table if exists test_null_cached") + sc.runSql("CREATE TABLE test_null_cached AS SELECT * FROM test_null") + + // clicks + sc.runSql("drop table if exists clicks") + sc.runSql("""create table clicks (id int, click int) + row format delimited fields terminated by '\t'""") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/clicks.txt' + OVERWRITE INTO TABLE clicks""") + sc.runSql("drop table if exists clicks_cached") + sc.runSql("create table clicks_cached as select * from clicks") + + // users + sc.runSql("drop table if exists users") + sc.runSql("""create table users (id int, name string) + row format delimited fields terminated by '\t'""") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/users.txt' + OVERWRITE INTO TABLE users""") + sc.runSql("drop table if exists users_cached") + sc.runSql("create table users_cached as select * from users") + + // test1 + sc.sql("drop table if exists test1") + sc.sql("""CREATE TABLE test1 (id INT, test1val ARRAY) + row format delimited fields terminated by '\t'""") + sc.sql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/test1.txt' INTO TABLE test1") + sc.sql("drop table if exists test1_cached") + sc.sql("CREATE TABLE test1_cached AS SELECT * FROM test1") + } + + def expectSql(sql: String, expectedResults: Array[String], sort: Boolean = true) { + val sharkResults: Array[String] = sc.runSql(sql).results.map(_.mkString("\t")).toArray + val results = if (sort) sharkResults.sortWith(_ < _) else sharkResults + val expected = if (sort) expectedResults.sortWith(_ < _) else expectedResults + assert(results.corresponds(expected)(_.equals(_)), + "In SQL: " + sql + "\n" + + "Expected: " + expected.mkString("\n") + "; got " + results.mkString("\n")) + } + + // A shortcut for single row results. + def expectSql(sql: String, expectedResult: String) { + expectSql(sql, Array(expectedResult)) + } + +} diff --git a/src/test/scala/shark/SortSuite.scala b/src/test/scala/shark/SortSuite.scala index 4e7e9c05..df948a54 100644 --- a/src/test/scala/shark/SortSuite.scala +++ b/src/test/scala/shark/SortSuite.scala @@ -31,28 +31,23 @@ class SortSuite extends FunSuite { TestUtils.init() + var sc: SparkContext = SharkRunner.init() + test("order by limit") { - var sc: SparkContext = null - try { - sc = new SparkContext("local", "test") - val data = Array((4, 14), (1, 11), (7, 17), (0, 10)) - val expected = data.sortWith(_._1 < _._1).toSeq - val rdd: RDD[(ReduceKey, BytesWritable)] = sc.parallelize(data, 50).map { x => - (new ReduceKeyMapSide(new BytesWritable(Array[Byte](x._1.toByte))), - new BytesWritable(Array[Byte](x._2.toByte))) - } - for (k <- 0 to 5) { - val sortedRdd = RDDUtils.topK(rdd, k).asInstanceOf[RDD[(ReduceKeyReduceSide, Array[Byte])]] - val output = sortedRdd.map { case(k, v) => - (k.byteArray(0).toInt, v(0).toInt) - }.collect().toSeq - assert(output.size === math.min(k, 4)) - assert(output === expected.take(math.min(k, 4))) - } - } finally { - sc.stop() + val data = Array((4, 14), (1, 11), (7, 17), (0, 10)) + val expected = data.sortWith(_._1 < _._1).toSeq + val rdd: RDD[(ReduceKey, BytesWritable)] = sc.parallelize(data, 50).map { x => + (new ReduceKeyMapSide(new BytesWritable(Array[Byte](x._1.toByte))), + new BytesWritable(Array[Byte](x._2.toByte))) + } + for (k <- 0 to 5) { + val sortedRdd = RDDUtils.topK(rdd, k).asInstanceOf[RDD[(ReduceKeyReduceSide, Array[Byte])]] + val output = sortedRdd.map { case(k, v) => + (k.byteArray(0).toInt, v(0).toInt) + }.collect().toSeq + assert(output.size === math.min(k, 4)) + assert(output === expected.take(math.min(k, 4))) } - sc.stop() - System.clearProperty("spark.driver.port") } + } From ab87ab3c03a10d1dc69a4b5ea6e9c7601d7998c5 Mon Sep 17 00:00:00 2001 From: Harvey Date: Thu, 9 Jan 2014 21:25:16 -0800 Subject: [PATCH 305/331] Support stats for partitioned tables, take 2 --- .../shark/execution/MapSplitPruning.scala | 21 +++- .../shark/execution/MapSplitPruningUtil.scala | 33 +++++++ .../scala/shark/execution/TableReader.scala | 98 ++++++++++++------- .../shark/execution/TableScanOperator.scala | 28 +++--- .../scala/shark/ColumnStatsSQLSuite.scala | 35 +++++++ 5 files changed, 163 insertions(+), 52 deletions(-) create mode 100644 src/main/scala/shark/execution/MapSplitPruningUtil.scala create mode 100644 src/test/scala/shark/ColumnStatsSQLSuite.scala diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index 584ad807..4de38fc0 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -17,6 +17,8 @@ package org.apache.hadoop.hive.ql.exec +import org.apache.hadoop.hive.serde2.objectinspector.{MapSplitPruningHelper, StructField} +import org.apache.hadoop.hive.serde2.objectinspector.UnionStructObjectInspector.MyField import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn @@ -85,7 +87,7 @@ object MapSplitPruning { columnEval: ExprNodeColumnEvaluator, expEvals: Array[ExprNodeEvaluator]): Boolean = { - val field = columnEval.field.asInstanceOf[IDStructField] + val field = getIDStructField(columnEval.field) val columnStats = s.stats(field.fieldID) if (columnStats != null) { @@ -107,7 +109,7 @@ object MapSplitPruning { leftEval: ExprNodeConstantEvaluator, rightEval: ExprNodeConstantEvaluator): Boolean = { - val field = columnEval.field.asInstanceOf[IDStructField] + val field = getIDStructField(columnEval.field) val columnStats = s.stats(field.fieldID) val leftValue: Object = leftEval.expr.getValue val rightValue: Object = rightEval.expr.getValue @@ -156,7 +158,7 @@ object MapSplitPruning { if (columnEval != null && constEval != null) { // We can prune the partition only if it is a predicate of form // column op const, where op is <, >, =, <=, >=, !=. - val field = columnEval.field.asInstanceOf[IDStructField] + val field = getIDStructField(columnEval.field) val value: Object = constEval.expr.getValue val columnStats = s.stats(field.fieldID) @@ -178,4 +180,17 @@ object MapSplitPruning { true } } + + private def getIDStructField(field: StructField): IDStructField = { + field match { + case myField: MyField => { + MapSplitPruningHelper.getStructFieldFromUnionOIField(myField) + .asInstanceOf[IDStructField] + } + case idStructField: IDStructField => idStructField + case otherFieldType: Any => { + throw new Exception("Unrecognized StructField: " + otherFieldType) + } + } + } } diff --git a/src/main/scala/shark/execution/MapSplitPruningUtil.scala b/src/main/scala/shark/execution/MapSplitPruningUtil.scala new file mode 100644 index 00000000..35a1041b --- /dev/null +++ b/src/main/scala/shark/execution/MapSplitPruningUtil.scala @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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.hadoop.hive.serde2.objectinspector + +import org.apache.hadoop.hive.serde2.objectinspector.UnionStructObjectInspector.MyField + + +object MapSplitPruningHelper { + + /** + * Extract the UnionStructObjectInspector.MyField's `structField` reference, which is + * package-private. + */ + def getStructFieldFromUnionOIField(unionOIMyField: MyField): StructField = { + unionOIMyField.structField + } + +} diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 56ea5b23..d9993baa 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -48,14 +48,17 @@ trait TableReader extends LogHelper { trait MemoryTableReader extends TableReader { - def makeRDDForTableWithStats( - hiveTable: HiveTable - ): (RDD[_], collection.Map[Int, TablePartitionStats]) + type PruningFunctionType = (RDD[_], collection.Map[Int, TablePartitionStats]) => RDD[_] - def makeRDDForPartitionedTableWithStats( - partitions: Seq[HivePartition] - ): (RDD[_], collection.Map[Int, TablePartitionStats]) + def makeRDDForTable( + hiveTable: HiveTable, + pruningFnOpt: Option[PruningFunctionType] + ): RDD[_] + def makeRDDForPartitionedTable( + partitions: Seq[HivePartition], + pruningFnOpt: Option[PruningFunctionType] + ): RDD[_] } /** Helper class for scanning tables stored in Tachyon. */ @@ -67,12 +70,13 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends MemoryTableRe private val _tableName = _tableNameSplit(1) override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { - makeRDDForTableWithStats(hiveTable)._1 + makeRDDForTable(hiveTable, pruningFnOpt = None) } - override def makeRDDForTableWithStats( - hiveTable: HiveTable - ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { + override def makeRDDForTable( + hiveTable: HiveTable, + pruningFnOpt: Option[PruningFunctionType] + ): RDD[_] = { // Table is in Tachyon. val tableKey = SharkEnv.makeTachyonTableKey(_databaseName, _tableName) if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { @@ -82,16 +86,23 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends MemoryTableRe val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( statsByteBuffer.array()) logInfo("Fetching table " + tableKey + " stats from Tachyon.") - (SharkEnv.tachyonUtil.createRDD(tableKey), indexToStats) + val (tableRdd, tableStats) = (SharkEnv.tachyonUtil.createRDD(tableKey), indexToStats) + if (pruningFnOpt.isDefined) { + val pruningFn = pruningFnOpt.get + pruningFn(tableRdd, tableStats) + } else { + tableRdd + } } override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") } - override def makeRDDForPartitionedTableWithStats( - partitions: Seq[HivePartition] - ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { + override def makeRDDForPartitionedTable( + partitions: Seq[HivePartition], + pruningFnOpt: Option[PruningFunctionType] + ): RDD[_] = { throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") } @@ -105,10 +116,11 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade private val _databaseName = _tableNameSplit(0) private val _tableName = _tableNameSplit(1) - /** Fetches the RDD for `_tableName` from the Shark metastore. */ - override def makeRDDForTableWithStats( - hiveTable: HiveTable - ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { + /** Fetches and optionally prunes the RDD for `_tableName` from the Shark metastore. */ + override def makeRDDForTable( + hiveTable: HiveTable, + pruningFnOpt: Option[PruningFunctionType] + ): RDD[_] = { logInfo("Loading table %s.%s from Spark block manager".format(_databaseName, _tableName)) val tableOpt = SharkEnv.memoryMetadataManager.getMemoryTable(_databaseName, _tableName) if (tableOpt.isEmpty) { @@ -116,24 +128,31 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } val table = tableOpt.get - (table.getRDD.get, table.getStats.get) + val (tableRdd, tableStats) = (table.getRDD.get, table.getStats.get) + // Prune if an applicable function is given. + if (pruningFnOpt.isDefined) { + val pruningFn = pruningFnOpt.get + pruningFn(tableRdd, tableStats) + } else { + tableRdd + } } override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { - makeRDDForTableWithStats(hiveTable)._1 + makeRDDForTable(hiveTable, pruningFnOpt = None) } /** - * Fetch an RDD from the Shark metastore using each partition key given, and return a union of all - * the fetched RDDs. + * Fetch and optinally prune an RDD from the Shark metastore using each partition key given, and + * return a union of all the fetched (and possibly pruned) RDDs. * * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. */ - override def makeRDDForPartitionedTableWithStats( - partitions: Seq[HivePartition] - ): (RDD[_], collection.Map[Int, TablePartitionStats]) = { - val unionedStats = new ArrayBuffer[(Int, TablePartitionStats)] + override def makeRDDForPartitionedTable( + partitions: Seq[HivePartition], + pruningFnOpt: Option[PruningFunctionType] + ): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) // Get partition field info @@ -161,14 +180,27 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade val rddAndStatsOpt = hivePartitionedTable.getPartitionAndStats(partitionKeyStr) if (rddAndStatsOpt.isEmpty) throwMissingPartitionException(partitionKeyStr) val (hivePartitionRDD, hivePartitionStats) = (rddAndStatsOpt.get._1, rddAndStatsOpt.get._2) - // Union any stats previously collected. - Table.mergeStats(unionedStats, hivePartitionStats) + val prunedPartitionRDD = if (pruningFnOpt.isDefined) { + val pruningFn = pruningFnOpt.get + pruningFn(hivePartitionRDD, hivePartitionStats) + } else { + hivePartitionRDD + } hivePartitionRDD.mapPartitions { iter => if (iter.hasNext) { + val nextElem = iter.next() + // `pruningFn` may or may not unravel the TablePartition. + val rowIter = nextElem match { + case tablePartition: TablePartition => { + tablePartition.iterator + } + case _ => { + iter + } + } // Map each tuple to a row object val rowWithPartArr = new Array[Object](2) - val tablePartition: TablePartition = iter.next() - tablePartition.iterator.map { value => + rowIter.map { value => rowWithPartArr.update(0, value.asInstanceOf[Object]) rowWithPartArr.update(1, partValues) rowWithPartArr.asInstanceOf[Object] @@ -179,14 +211,14 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } } if (hivePartitionRDDs.size > 0) { - (new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs), unionedStats.toMap) + new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs) } else { - (new EmptyRDD[Object](SharkEnv.sc), new HashMap[Int, TablePartitionStats]) + new EmptyRDD[Object](SharkEnv.sc) } } override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { - makeRDDForPartitionedTableWithStats(partitions)._1 + makeRDDForPartitionedTable(partitions, pruningFnOpt = None) } /** diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 5be02981..74c3161d 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -117,33 +117,28 @@ class TableScanOperator extends TopOperator[TableScanDesc] { tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. if (isInMemoryTableScan) { - assert (cacheMode == CacheType.MEMORY || cacheMode == CacheType.MEMORY_ONLY, + assert ( + cacheMode == CacheType.MEMORY || cacheMode == CacheType.MEMORY_ONLY || + cacheMode == CacheType.TACHYON, "Table %s.%s is in Shark metastore, but its cacheMode (%s) indicates otherwise". format(databaseName, tableName, cacheMode)) - val tableReader = new HeapTableReader(tableDesc) - if (table.isPartitioned) { - return tableReader.makeRDDForPartitionedTable(parts) + val tableReader = if (cacheMode == CacheType.TACHYON) { + new TachyonTableReader(tableDesc) } else { - val (tableRdd, stats) = tableReader.makeRDDForTableWithStats(table) - return createPrunedRdd(databaseName, tableName, tableRdd, stats) + new HeapTableReader(tableDesc) } - } else if (cacheMode == CacheType.TACHYON) { - val tableReader = new TachyonTableReader(tableDesc) if (table.isPartitioned) { - return tableReader.makeRDDForPartitionedTable(parts) + tableReader.makeRDDForPartitionedTable(parts, Some(createPrunedRdd _)) } else { - val (tableRdd, stats) = tableReader.makeRDDForTableWithStats(table) - return createPrunedRdd(databaseName, tableName, tableRdd, stats) + tableReader.makeRDDForTable(table, Some(createPrunedRdd _)) } } else { // Table is a Hive table on HDFS (or other Hadoop storage). - return makeRDDFromHadoop() + makeRDDFromHadoop() } } private def createPrunedRdd( - databaseName: String, - tableName: String, rdd: RDD[_], indexToStats: collection.Map[Int, TablePartitionStats]): RDD[_] = { // Run map pruning if the flag is set, there exists a filter predicate on @@ -182,7 +177,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { val prunedRdd = PartitionPruningRDD.create(rdd, prunePartitionFunc) val timeTaken = System.currentTimeMillis - startTime logInfo("Map pruning %d partitions into %s partitions took %d ms".format( - rdd.partitions.size, prunedRdd.partitions.size, timeTaken)) + rdd.partitions.size, prunedRdd.partitions.size, timeTaken)) prunedRdd } else { rdd @@ -190,7 +185,8 @@ class TableScanOperator extends TopOperator[TableScanDesc] { return prunedRdd.mapPartitions { iter => if (iter.hasNext) { - val tablePartition = iter.next().asInstanceOf[TablePartition] + val tablePartition1 = iter.next() + val tablePartition = tablePartition1.asInstanceOf[TablePartition] tablePartition.prunedIterator(columnsUsed) } else { Iterator.empty diff --git a/src/test/scala/shark/ColumnStatsSQLSuite.scala b/src/test/scala/shark/ColumnStatsSQLSuite.scala new file mode 100644 index 00000000..10ffdad9 --- /dev/null +++ b/src/test/scala/shark/ColumnStatsSQLSuite.scala @@ -0,0 +1,35 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import org.apache.hadoop.io.BytesWritable + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD + + +class ColumnStatsSQLSuite extends FunSuite { + + var sc: SparkContext = SharkRunner.init() + + test("A Wild Sheep Chase") { + assert(true) + } +} From 105c84338f5e54a5ab57843c89c3da2df48cc034 Mon Sep 17 00:00:00 2001 From: Harvey Date: Fri, 10 Jan 2014 11:25:03 -0800 Subject: [PATCH 306/331] Simple test suite for stats collection and pruning for partitioned tables --- .../scala/shark/ColumnStatsSQLSuite.scala | 104 +++++++++++++++++- src/test/scala/shark/SQLSuite.scala | 29 +---- 2 files changed, 102 insertions(+), 31 deletions(-) diff --git a/src/test/scala/shark/ColumnStatsSQLSuite.scala b/src/test/scala/shark/ColumnStatsSQLSuite.scala index 10ffdad9..f0aa5931 100644 --- a/src/test/scala/shark/ColumnStatsSQLSuite.scala +++ b/src/test/scala/shark/ColumnStatsSQLSuite.scala @@ -19,17 +19,111 @@ package shark import org.apache.hadoop.io.BytesWritable +import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import org.apache.spark.SparkContext +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME + import org.apache.spark.rdd.RDD +import shark.memstore2.MemoryMetadataManager + + +class ColumnStatsSQLSuite extends FunSuite with BeforeAndAfterAll { + + val sc: SharkContext = SharkRunner.init() + val sharkMetastore = SharkEnv.memoryMetadataManager + + // import expectSql() shortcut methods + import shark.SharkRunner._ + + override def beforeAll() { + sc.runSql("drop table if exists srcpart_cached") + sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + into table srcpart_cached partition (keypart = 1)""") + } + + override def afterAll() { + sc.runSql("drop table if exists srcpart_cached") + } + + test("Hive partition stats are tracked") { + val tableOpt = sharkMetastore.getPartitionedTable(DEFAULT_DATABASE_NAME, "srcpart_cached") + assert(tableOpt.isDefined) + val partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + val partitionToStats = partitionToStatsOpt.get + // The 'kv1.txt' file loaded into 'keypart=1' in beforeAll() has 2 partitions. + assert(partitionToStats.size == 2) + } + + test("Hive partition stats are tracked after LOADs and INSERTs") { + // Load more data into srcpart_cached + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + into table srcpart_cached partition (keypart = 1)""") + val tableOpt = sharkMetastore.getPartitionedTable(DEFAULT_DATABASE_NAME, "srcpart_cached") + assert(tableOpt.isDefined) + var partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + var partitionToStats = partitionToStatsOpt.get + // The 'kv1.txt' file loaded into 'keypart=1' has 2 partitions. We've loaded it twice at this + // point. + assert(partitionToStats.size == 4) + + // Append using INSERT command + sc.runSql("insert into table srcpart_cached partition(keypart = 1) select * from test") + partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + partitionToStats = partitionToStatsOpt.get + assert(partitionToStats.size == 6) + + // INSERT OVERWRITE should overrwritie old table stats. This also restores srcpart_cached + // to contents contained before this test. + sc.runSql("""insert overwrite table srcpart_cached partition(keypart = 1) + select * from test""") + partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + partitionToStats = partitionToStatsOpt.get + assert(partitionToStats.size == 2) + } + + ////////////////////////////////////////////////////////////////////////////// + // End-to-end sanity checks + ////////////////////////////////////////////////////////////////////////////// + test("column pruning filters") { + expectSql("select count(*) from test_cached where key > -1", "500") + } + + test("column pruning group by") { + expectSql("select key, count(*) from test_cached group by key order by key limit 1", "0\t3") + } + + test("column pruning group by with single filter") { + expectSql("select key, count(*) from test_cached where val='val_484' group by key", "484\t1") + } -class ColumnStatsSQLSuite extends FunSuite { + test("column pruning aggregate function") { + expectSql("select val, sum(key) from test_cached group by val order by val desc limit 1", + "val_98\t196") + } + + test("column pruning filters for a Hive partition") { + expectSql("select count(*) from srcpart_cached where key > -1", "500") + expectSql("select count(*) from srcpart_cached where key > -1 and keypart = 1", "500") + } - var sc: SparkContext = SharkRunner.init() + test("column pruning group by for a Hive partition") { + expectSql("select key, count(*) from srcpart_cached group by key order by key limit 1", "0\t3") + } - test("A Wild Sheep Chase") { - assert(true) + test("column pruning group by with single filter for a Hive partition") { + expectSql("select key, count(*) from srcpart_cached where val='val_484' group by key", "484\t1") } + + test("column pruning aggregate function for a Hive partition") { + expectSql("select val, sum(key) from srcpart_cached group by val order by val desc limit 1", + "val_98\t196") + } + } diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index aff0f392..222057b3 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -40,12 +40,6 @@ class SQLSuite extends FunSuite { var sc: SharkContext = SharkRunner.init() var sharkMetastore: MemoryMetadataManager = SharkEnv.memoryMetadataManager - // Shortcut methods to cut down on line length - def expectSql(sql: String, expectedResults: Array[String], sort: Boolean = true) = - SharkRunner.expectSql(sql, expectedResults, sort) - - def expectSql(sql: String, expectedResults: String) = SharkRunner.expectSql(sql, expectedResults) - private def createCachedPartitionedTable( tableName: String, numPartitionsToCreate: Int, @@ -108,6 +102,9 @@ class SQLSuite extends FunSuite { assert(diskSum == cacheSum, "Sum of keys from cached and disk contents differ") } + // import expectSql() shortcut methods + import shark.SharkRunner._ + ////////////////////////////////////////////////////////////////////////////// // basic SQL ////////////////////////////////////////////////////////////////////////////// @@ -174,26 +171,6 @@ class SQLSuite extends FunSuite { sort = false) } - ////////////////////////////////////////////////////////////////////////////// - // column pruning - ////////////////////////////////////////////////////////////////////////////// - test("column pruning filters") { - expectSql("select count(*) from test_cached where key > -1", "500") - } - - test("column pruning group by") { - expectSql("select key, count(*) from test_cached group by key order by key limit 1", "0\t3") - } - - test("column pruning group by with single filter") { - expectSql("select key, count(*) from test_cached where val='val_484' group by key", "484\t1") - } - - test("column pruning aggregate function") { - expectSql("select val, sum(key) from test_cached group by val order by val desc limit 1", - "val_98\t196") - } - ////////////////////////////////////////////////////////////////////////////// // join ////////////////////////////////////////////////////////////////////////////// From d5d0c2d16a93397fe17a7eceb07210b904bc258c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Tue, 7 Jan 2014 15:11:04 -0800 Subject: [PATCH 307/331] Accept v0.8.0 CacheType enum names. --- src/main/scala/shark/memstore2/CacheType.scala | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala index 6115c542..ed1e1735 100644 --- a/src/main/scala/shark/memstore2/CacheType.scala +++ b/src/main/scala/shark/memstore2/CacheType.scala @@ -17,10 +17,13 @@ package shark.memstore2 +import shark.LogHelper + + /* * Enumerations and static helper functions for caches supported by Shark. */ -object CacheType extends Enumeration { +object CacheType extends Enumeration with LogHelper { /* * The CacheTypes: @@ -44,8 +47,14 @@ object CacheType extends Enumeration { MEMORY } else { try { - // Try to use Scala's Enumeration::withName() to interpret 'name'. - withName(name.toUpperCase) + if (name.toUpperCase == "HEAP") { + // Interpret 'HEAP' as 'MEMORY' to ensure backwards compatibility with Shark 0.8.0. + logWarning("The 'HEAP' cache type name is deprecated. Use 'MEMORY' instead.") + MEMORY + } else { + // Try to use Scala's Enumeration::withName() to interpret 'name'. + withName(name.toUpperCase) + } } catch { case e: java.util.NoSuchElementException => throw new InvalidCacheTypeException(name) } @@ -53,5 +62,5 @@ object CacheType extends Enumeration { } class InvalidCacheTypeException(name: String) - extends Exception("Invalid string representation of cache type " + name) + extends Exception("Invalid string representation of cache type: '%s'".format(name)) } From 080589a7124ddff1bffe639ce7aa0fd75f7c9b03 Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 14 Jan 2014 13:00:21 -0800 Subject: [PATCH 308/331] Patches for partition stats and pruning --- ...Util.scala => MapSplitPruningHelper.scala} | 0 .../scala/shark/execution/TableReader.scala | 22 ++++++------------- .../execution/optimization/ColumnPruner.scala | 3 ++- 3 files changed, 9 insertions(+), 16 deletions(-) rename src/main/scala/shark/execution/{MapSplitPruningUtil.scala => MapSplitPruningHelper.scala} (100%) diff --git a/src/main/scala/shark/execution/MapSplitPruningUtil.scala b/src/main/scala/shark/execution/MapSplitPruningHelper.scala similarity index 100% rename from src/main/scala/shark/execution/MapSplitPruningUtil.scala rename to src/main/scala/shark/execution/MapSplitPruningHelper.scala diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index d9993baa..85a6b913 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -143,8 +143,8 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } /** - * Fetch and optinally prune an RDD from the Shark metastore using each partition key given, and - * return a union of all the fetched (and possibly pruned) RDDs. + * Fetches an RDD from the Shark metastore for each partition key given. Returns a single, unioned + * RDD representing all of the specified partition keys. * * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. @@ -178,7 +178,9 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade val hivePartitionedTable = hivePartitionedTableOpt.get val rddAndStatsOpt = hivePartitionedTable.getPartitionAndStats(partitionKeyStr) - if (rddAndStatsOpt.isEmpty) throwMissingPartitionException(partitionKeyStr) + if (rddAndStatsOpt.isEmpty) { + throwMissingPartitionException(partitionKeyStr) + } val (hivePartitionRDD, hivePartitionStats) = (rddAndStatsOpt.get._1, rddAndStatsOpt.get._2) val prunedPartitionRDD = if (pruningFnOpt.isDefined) { val pruningFn = pruningFnOpt.get @@ -186,21 +188,11 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } else { hivePartitionRDD } - hivePartitionRDD.mapPartitions { iter => + prunedPartitionRDD.mapPartitions { iter => if (iter.hasNext) { - val nextElem = iter.next() - // `pruningFn` may or may not unravel the TablePartition. - val rowIter = nextElem match { - case tablePartition: TablePartition => { - tablePartition.iterator - } - case _ => { - iter - } - } // Map each tuple to a row object val rowWithPartArr = new Array[Object](2) - rowIter.map { value => + iter.map { value => rowWithPartArr.update(0, value.asInstanceOf[Object]) rowWithPartArr.update(1, partValues) rowWithPartArr.asInstanceOf[Object] diff --git a/src/main/scala/shark/execution/optimization/ColumnPruner.scala b/src/main/scala/shark/execution/optimization/ColumnPruner.scala index 06ce0186..38efb328 100644 --- a/src/main/scala/shark/execution/optimization/ColumnPruner.scala +++ b/src/main/scala/shark/execution/optimization/ColumnPruner.scala @@ -37,7 +37,8 @@ class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends val columnsUsed: BitSet = { val colsToKeep = computeColumnsToKeep() - val allColumns = tbl.getAllCols().map(x => x.getName()) + // No need to prune partition columns - Hive does that for us. + val allColumns = tbl.getCols().map(x => x.getName()) val b = new BitSet() for (i <- Range(0, allColumns.size) if colsToKeep.contains(allColumns(i))) { b.set(i, true) From c9199cd8343302b05ac1d9a0613711446e62ca46 Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 14 Jan 2014 13:06:12 -0800 Subject: [PATCH 309/331] Tachyon support Includes - LOAD, INSERT, ALTER commands already supported on heap tables - Partitioned tables --- src/main/scala/shark/SharkEnv.scala | 28 ----- .../execution/MemoryStoreSinkOperator.scala | 82 +++++++------ .../shark/execution/OperatorFactory.scala | 4 +- .../scala/shark/execution/SharkDDLTask.scala | 79 +++++++++---- .../scala/shark/execution/SparkLoadTask.scala | 82 +++++++++---- .../scala/shark/execution/TableReader.scala | 85 ++++++++++---- .../shark/execution/TableScanOperator.scala | 27 +++-- .../memstore2/LazySimpleSerDeWrapper.scala | 49 ++++++++ .../memstore2/MemoryMetadataManager.scala | 84 +++++++------- .../memstore2/PartitionedMemoryTable.scala | 14 ++- .../scala/shark/memstore2/TableRecovery.scala | 7 +- .../parse/SharkDDLSemanticAnalyzer.scala | 57 +++++---- .../parse/SharkLoadSemanticAnalyzer.scala | 11 +- .../shark/parse/SharkSemanticAnalyzer.scala | 50 ++++---- .../scala/shark/tachyon/TachyonUtil.scala | 23 ++-- .../scala/shark/tachyon/TachyonUtilImpl.scala | 30 +++-- .../scala/shark/tachyon/TachyonUtilImpl.scala | 108 ++++++++++++++---- 17 files changed, 540 insertions(+), 280 deletions(-) create mode 100644 src/main/scala/shark/memstore2/LazySimpleSerDeWrapper.scala diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index ee634399..65d56bd7 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -111,26 +111,6 @@ object SharkEnv extends LogHelper { val addedFiles = HashSet[String]() val addedJars = HashSet[String]() - /** - * Drops the table associated with 'key'. This method checks for Tachyon tables before - * delegating to MemoryMetadataManager#removeTable() for removing the table's entry from the - * Shark metastore. - * - * @param tableName The table that should be dropped from the Shark metastore and from memory - * storage. - */ - def dropTable(databaseName: String, tableName: String): Option[RDD[_]] = { - val tableKey = makeTachyonTableKey(databaseName, tableName) - if (SharkEnv.tachyonUtil.tachyonEnabled() && SharkEnv.tachyonUtil.tableExists(tableKey)) { - if (SharkEnv.tachyonUtil.dropTable(tableKey)) { - logInfo("Table " + tableKey + " was deleted from Tachyon."); - } else { - logWarning("Failed to remove table " + tableKey + " from Tachyon."); - } - } - memoryMetadataManager.removeTable(databaseName, tableName) - } - /** Cleans up and shuts down the Shark environments. */ def stop() { logDebug("Shutting down Shark Environment") @@ -145,14 +125,6 @@ object SharkEnv extends LogHelper { /** Return the value of an environmental variable as a string. */ def getEnv(varname: String) = if (System.getenv(varname) == null) "" else System.getenv(varname) - /** - * Return an identifier for RDDs that back tables stored in Tachyon. The format is - * "databaseName.tableName". - */ - def makeTachyonTableKey(databaseName: String, tableName: String): String = { - (databaseName + "." + tableName).toLowerCase - } - } diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index bf04af6c..06bb1cdb 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -46,7 +46,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // If true, columnar storage will use compression. @BeanProperty var shouldCompress: Boolean = _ - // For CTAS, this is the name of the table that is created. For INSERTS, this is the name of + // For CTAS, this is the name of the table that is created. For INSERTS, this is the name of* // the table that is modified. @BeanProperty var tableName: String = _ @@ -56,7 +56,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Used only for commands that target Hive partitions. The partition key is a set of unique values // for the the table's partitioning columns and identifies the partition (represented by an RDD) // that will be created or modified by the INSERT command being handled. - @BeanProperty var hivePartitionKey: String = _ + @BeanProperty var hivePartitionKeyOpt: Option[String] = _ // The memory storage used to store the output RDD - e.g., CacheType.HEAP refers to Spark's // block manager. @@ -87,11 +87,18 @@ class MemoryStoreSinkOperator extends TerminalOperator { val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) val op = OperatorSerializationWrapper(this) + val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( + databaseName, tableName) + val tableKey = MemoryMetadataManager.makeTableKey(databaseName, tableName) val tachyonWriter: TachyonTableWriter = if (cacheMode == CacheType.TACHYON) { + if (!useUnionRDD && SharkEnv.tachyonUtil.tableExists(tableKey, hivePartitionKeyOpt)) { + // For INSERT OVERWRITE, delete the old table or Hive partition directory, if it exists. + SharkEnv.tachyonUtil.dropTable(tableKey, hivePartitionKeyOpt) + } // Use an additional row to store metadata (e.g. number of rows in each partition). - SharkEnv.tachyonUtil.createTableWriter(tableName, numColumns + 1) + SharkEnv.tachyonUtil.createTableWriter(tableKey, hivePartitionKeyOpt, numColumns + 1) } else { null } @@ -120,12 +127,9 @@ class MemoryStoreSinkOperator extends TerminalOperator { } } - val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( - databaseName, tableName) - // If true, a UnionRDD will be used to combine the RDD that contains the query output with the // previous RDD, which is fetched using 'tableName' or - if the table is Hive-partitioned - a - // ('tableName', 'hivePartitionKey') pair. + // ('tableName', 'hivePartitionKeyOpt.get') pair. var hasPreviousRDDForUnion = false if (tachyonWriter != null) { @@ -161,38 +165,42 @@ class MemoryStoreSinkOperator extends TerminalOperator { tableName, if (cacheMode == CacheType.NONE) "disk" else cacheMode.toString)) - val tableStats = if (isHivePartitioned) { - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - databaseName, tableName).get - outputRDD.setName("%s.%s(%s)".format(databaseName, tableName, hivePartitionKey)) - if (useUnionRDD && hasPreviousRDDForUnion) { - // An RDD for the Hive partition already exists, so update its metadata entry in - // 'partitionedTable'. - assert(outputRDD.isInstanceOf[UnionRDD[_]]) - partitionedTable.updatePartition(hivePartitionKey, outputRDD, statsAcc.value) - } else { - // This is a new Hive-partition. Add a new metadata entry in 'partitionedTable'. - partitionedTable.putPartition(hivePartitionKey, outputRDD, statsAcc.value.toMap) - } - // Stats should be updated at this point. - partitionedTable.getStats(hivePartitionKey).get - } else { - outputRDD.setName(tableName) - // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). - val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) - .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( - databaseName, tableName, cacheMode)) - if (useUnionRDD) { - memoryTable.update(outputRDD, statsAcc.value) - } else { - memoryTable.put(outputRDD, statsAcc.value.toMap) - } - // TODO: For Tachyon support, have Shark Tables manage stats store and updates. - if (tachyonWriter != null) { + val tableStats = + if (cacheMode == CacheType.TACHYON) { tachyonWriter.updateMetadata(ByteBuffer.wrap(JavaSerializer.serialize(statsAcc.value.toMap))) + statsAcc.value.toMap + } else { + if (isHivePartitioned) { + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + databaseName, tableName).get + val hivePartitionKey = hivePartitionKeyOpt.get + outputRDD.setName("%s.%s(%s)".format(databaseName, tableName, hivePartitionKey)) + if (useUnionRDD && hasPreviousRDDForUnion) { + // An RDD for the Hive partition already exists, so update its metadata entry in + // 'partitionedTable'. + assert(outputRDD.isInstanceOf[UnionRDD[_]]) + partitionedTable.updatePartition(hivePartitionKey, outputRDD, statsAcc.value) + } else { + // This is a new Hive-partition. Add a new metadata entry in 'partitionedTable'. + partitionedTable.putPartition(hivePartitionKey, outputRDD, statsAcc.value.toMap) + } + // Stats should be updated at this point. + partitionedTable.getStats(hivePartitionKey).get + } else { + outputRDD.setName(tableName) + // Create a new MemoryTable entry if one doesn't exist (i.e., this operator is for a CTAS). + val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) + .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( + databaseName, tableName, cacheMode)) + if (useUnionRDD) { + // Ok, a Tachyon table should manage stats for each rdd, and never union the maps. + memoryTable.update(outputRDD, statsAcc.value) + } else { + memoryTable.put(outputRDD, statsAcc.value.toMap) + } + memoryTable.getStats.get + } } - memoryTable.getStats.get - } if (SharkConfVars.getBoolVar(localHconf, SharkConfVars.MAP_PRUNING_PRINT_DEBUG)) { tableStats.foreach { case(index, tablePartitionStats) => diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index a6c9323c..e562020e 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -47,7 +47,7 @@ object OperatorFactory extends LogHelper { tableName: String, databaseName: String, numColumns: Int, - hivePartitionKey: String, + hivePartitionKeyOpt: Option[String], cacheMode: CacheType, useUnionRDD: Boolean): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? @@ -59,7 +59,7 @@ object OperatorFactory extends LogHelper { sinkOp.databaseName = databaseName sinkOp.numColumns = numColumns sinkOp.cacheMode = cacheMode - sinkOp.hivePartitionKey = hivePartitionKey + sinkOp.hivePartitionKeyOpt = hivePartitionKeyOpt sinkOp.useUnionRDD = useUnionRDD _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } diff --git a/src/main/scala/shark/execution/SharkDDLTask.scala b/src/main/scala/shark/execution/SharkDDLTask.scala index ddbb8373..21707ed3 100644 --- a/src/main/scala/shark/execution/SharkDDLTask.scala +++ b/src/main/scala/shark/execution/SharkDDLTask.scala @@ -30,15 +30,15 @@ import org.apache.hadoop.hive.ql.plan.api.StageType import org.apache.spark.rdd.EmptyRDD import shark.{LogHelper, SharkEnv} -import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} +import shark.memstore2.{CacheType, MemoryTable, MemoryMetadataManager, PartitionedMemoryTable} import shark.memstore2.{SharkTblProperties, TablePartitionStats} import shark.util.HiveUtils private[shark] class SharkDDLWork(val ddlDesc: DDLDesc) extends java.io.Serializable { - // Used only for CREATE TABLE. var cacheMode: CacheType.CacheType = _ + } /** @@ -61,9 +61,9 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] // use. work.ddlDesc match { case creatTblDesc: CreateTableDesc => createTable(hiveDb, creatTblDesc, work.cacheMode) - case addPartitionDesc: AddPartitionDesc => addPartition(hiveDb, addPartitionDesc) - case dropTableDesc: DropTableDesc => dropTableOrPartition(hiveDb, dropTableDesc) - case alterTableDesc: AlterTableDesc => alterTable(hiveDb, alterTableDesc) + case addPartitionDesc: AddPartitionDesc => addPartition(hiveDb, addPartitionDesc, work.cacheMode) + case dropTableDesc: DropTableDesc => dropTableOrPartition(hiveDb, dropTableDesc, work.cacheMode) + case alterTableDesc: AlterTableDesc => alterTable(hiveDb, alterTableDesc, work.cacheMode) case _ => { throw new UnsupportedOperationException( "Shark does not require a Shark DDL task for: " + work.ddlDesc.getClass.getName) @@ -92,17 +92,23 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val tableName = createTblDesc.getTableName val tblProps = createTblDesc.getTblProps - val isHivePartitioned = (createTblDesc.getPartCols.size > 0) - if (isHivePartitioned) { - // Add a new PartitionedMemoryTable entry in the Shark metastore. - // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. - SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( - dbName, tableName, cacheMode, tblProps) + if (cacheMode == CacheType.TACHYON) { + // For Tachyon tables (partitioned or not), just create the parent directory. + SharkEnv.tachyonUtil.createDirectory( + MemoryMetadataManager.makeTableKey(dbName, tableName), hivePartitionKeyOpt = None) } else { - val memoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( - dbName, tableName, cacheMode) - // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. - memoryTable.put(new EmptyRDD(SharkEnv.sc)) + val isHivePartitioned = (createTblDesc.getPartCols.size > 0) + if (isHivePartitioned) { + // Add a new PartitionedMemoryTable entry in the Shark metastore. + // An empty table has a PartitionedMemoryTable entry with no 'hivePartition -> RDD' mappings. + SharkEnv.memoryMetadataManager.createPartitionedMemoryTable( + dbName, tableName, cacheMode, tblProps) + } else { + val memoryTable = SharkEnv.memoryMetadataManager.createMemoryTable( + dbName, tableName, cacheMode) + // An empty table has a MemoryTable table entry with 'tableRDD' referencing an EmptyRDD. + memoryTable.put(new EmptyRDD(SharkEnv.sc)) + } } } @@ -114,10 +120,10 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] */ def addPartition( hiveMetadataDb: Hive, - addPartitionDesc: AddPartitionDesc) { + addPartitionDesc: AddPartitionDesc, + cacheMode: CacheType.CacheType) { val dbName = hiveMetadataDb.getCurrentDatabase() val tableName = addPartitionDesc.getTableName - val partitionedTable = getPartitionedTableWithAssertions(dbName, tableName) // Find the set of partition column values that specifies the partition being added. val hiveTable = db.getTable(tableName, false /* throwException */); @@ -125,7 +131,13 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] val partColToValue: JavaMap[String, String] = addPartitionDesc.getPartSpec // String format for partition key: 'col1=value1/col2=value2/...' val partKeyStr: String = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partColToValue) - partitionedTable.putPartition(partKeyStr, new EmptyRDD(SharkEnv.sc)) + if (cacheMode == CacheType.TACHYON) { + SharkEnv.tachyonUtil.createDirectory( + MemoryMetadataManager.makeTableKey(dbName, tableName), Some(partKeyStr)) + } else { + val partitionedTable = getPartitionedTableWithAssertions(dbName, tableName) + partitionedTable.putPartition(partKeyStr, new EmptyRDD(SharkEnv.sc)) + } } /** @@ -138,25 +150,35 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] */ def dropTableOrPartition( hiveMetadataDb: Hive, - dropTableDesc: DropTableDesc) { + dropTableDesc: DropTableDesc, + cacheMode: CacheType.CacheType) { val dbName = hiveMetadataDb.getCurrentDatabase() val tableName = dropTableDesc.getTableName val hiveTable = db.getTable(tableName, false /* throwException */); val partSpecs: JavaList[PartitionSpec] = dropTableDesc.getPartSpecs + val tableKey = MemoryMetadataManager.makeTableKey(dbName, tableName) if (partSpecs == null) { // The command is a true DROP TABLE. - SharkEnv.dropTable(dbName, tableName) + if (cacheMode == CacheType.TACHYON) { + SharkEnv.tachyonUtil.dropTable(tableKey, hivePartitionKeyOpt = None) + } else { + SharkEnv.memoryMetadataManager.removeTable(dbName, tableName) + } } else { // The command is an ALTER TABLE DROP PARTITION - val partitionedTable = getPartitionedTableWithAssertions(dbName, tableName) // Find the set of partition column values that specifies the partition being dropped. val partCols: Seq[String] = hiveTable.getPartCols.map(_.getName) for (partSpec <- partSpecs) { val partColToValue: JavaMap[String, String] = partSpec.getPartSpecWithoutOperator // String format for partition key: 'col1=value1/col2=value2/...' val partKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partColToValue) - partitionedTable.removePartition(partKeyStr) + if (cacheMode == CacheType.TACHYON) { + SharkEnv.tachyonUtil.dropTable(tableKey, Some(partKeyStr)) + } else { + val partitionedTable = getPartitionedTableWithAssertions(dbName, tableName) + getPartitionedTableWithAssertions(dbName, tableName).removePartition(partKeyStr) + } } } } @@ -167,17 +189,24 @@ private[shark] class SharkDDLTask extends HiveTask[SharkDDLWork] * @param hiveMetadataDb Namespace of the table to update. * @param alterTableDesc Hive metadata object containing fields needed to handle various table * update commands, such as ALTER TABLE
RENAME TO. - * + * */ def alterTable( hiveMetadataDb: Hive, - alterTableDesc: AlterTableDesc) { + alterTableDesc: AlterTableDesc, + cacheMode: CacheType.CacheType) { val dbName = hiveMetadataDb.getCurrentDatabase() alterTableDesc.getOp() match { case AlterTableDesc.AlterTableTypes.RENAME => { val oldName = alterTableDesc.getOldName val newName = alterTableDesc.getNewName - SharkEnv.memoryMetadataManager.renameTable(dbName, oldName, newName) + if (cacheMode == CacheType.TACHYON) { + val oldTableKey = MemoryMetadataManager.makeTableKey(dbName, oldName) + val newTableKey = MemoryMetadataManager.makeTableKey(dbName, newName) + SharkEnv.tachyonUtil.renameDirectory(oldTableKey, newTableKey) + } else { + SharkEnv.memoryMetadataManager.renameTable(dbName, oldName, newName) + } } case _ => { // TODO(harvey): Support more ALTER TABLE commands, such as ALTER TABLE PARTITION RENAME TO. diff --git a/src/main/scala/shark/execution/SparkLoadTask.scala b/src/main/scala/shark/execution/SparkLoadTask.scala index 019d391a..e219fd6d 100644 --- a/src/main/scala/shark/execution/SparkLoadTask.scala +++ b/src/main/scala/shark/execution/SparkLoadTask.scala @@ -19,6 +19,7 @@ package shark.execution import java.io.Serializable +import java.nio.ByteBuffer import java.util.{Properties, Map => JavaMap} import scala.collection.JavaConversions._ @@ -41,8 +42,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import shark.{LogHelper, SharkEnv, Utils} -import shark.execution.serialization.KryoSerializer +import shark.execution.serialization.{KryoSerializer, JavaSerializer} import shark.memstore2._ +import shark.tachyon.TachyonTableWriter import shark.util.HiveUtils @@ -54,7 +56,6 @@ import shark.util.HiveUtils * @param commandType Enum representing the command that will be executed for the target table. See * SparkLoadWork.CommandTypes for a description of which SQL commands correspond to each type. * @param cacheMode Cache type that the RDD should be stored in (e.g., Spark heap). - * TODO(harvey): Support Tachyon. */ private[shark] class SparkLoadWork( @@ -186,15 +187,27 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe * @param broadcastedHiveConf Allows for sharing a Hive Configuration broadcast used to create * the Hadoop `inputRdd`. * @param inputOI Object inspector used to read rows from `inputRdd`. + * @param hivePartitionKeyOpt A defined Hive partition key if the RDD being loaded is part of a + * Hive-partitioned table. */ private def materialize( inputRdd: RDD[_], serDeProps: Properties, broadcastedHiveConf: Broadcast[SerializableWritable[HiveConf]], - inputOI: StructObjectInspector) = { + inputOI: StructObjectInspector, + tableKey: String, + hivePartitionKeyOpt: Option[String]) = { val statsAcc = SharkEnv.sc.accumulableCollection(ArrayBuffer[(Int, TablePartitionStats)]()) + val tachyonWriter = if (work.cacheMode == CacheType.TACHYON) { + // Find the number of columns in the table schema using `serDeProps`. + val numColumns = serDeProps.getProperty(Constants.LIST_COLUMNS).split(',').size + // Use an additional row to store metadata (e.g. number of rows in each partition). + SharkEnv.tachyonUtil.createTableWriter(tableKey, hivePartitionKeyOpt, numColumns + 1) + } else { + null + } val serializedOI = KryoSerializer.serialize(inputOI) - val transformedRdd = inputRdd.mapPartitionsWithIndex { case (partIndex, partIter) => + var transformedRdd = inputRdd.mapPartitionsWithIndex { case (partIndex, partIter) => val serde = new ColumnarSerDe serde.initialize(broadcastedHiveConf.value.value, serDeProps) val localInputOI = KryoSerializer.deserialize[ObjectInspector](serializedOI) @@ -212,9 +225,30 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe } } // Run a job to materialize the RDD. - transformedRdd.persist(StorageLevel.MEMORY_AND_DISK) + if (work.cacheMode == CacheType.TACHYON) { + // Put the table in Tachyon. + logInfo("Putting RDD for %s in Tachyon".format(tableKey)) + if (work.commandType == SparkLoadWork.CommandTypes.OVERWRITE && + SharkEnv.tachyonUtil.tableExists(tableKey, hivePartitionKeyOpt)) { + // For INSERT OVERWRITE, delete the old table or Hive partition directory, if it exists. + SharkEnv.tachyonUtil.dropTable(tableKey, hivePartitionKeyOpt) + } + tachyonWriter.createTable(ByteBuffer.allocate(0)) + transformedRdd = transformedRdd.mapPartitionsWithIndex { case(part, iter) => + val partition = iter.next() + partition.toTachyon.zipWithIndex.foreach { case(buf, column) => + tachyonWriter.writeColumnPartition(column, part, buf) + } + Iterator(partition) + } + } else { + transformedRdd.persist(StorageLevel.MEMORY_AND_DISK) + } transformedRdd.context.runJob( transformedRdd, (iter: Iterator[TablePartition]) => iter.foreach(_ => Unit)) + if (work.cacheMode == CacheType.TACHYON) { + tachyonWriter.updateMetadata(ByteBuffer.wrap(JavaSerializer.serialize(statsAcc.value.toMap))) + } (transformedRdd, statsAcc.value) } @@ -255,7 +289,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe pathFilterOpt: Option[PathFilter]) { val databaseName = hiveTable.getDbName val tableName = hiveTable.getTableName - val memoryTable = getOrCreateMemoryTable(hiveTable) val tableSchema = hiveTable.getSchema val serDe = hiveTable.getDeserializer serDe.initialize(conf, tableSchema) @@ -266,12 +299,17 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe inputRDD, tableSchema, hadoopReader.broadcastedHiveConf, - serDe.getObjectInspector.asInstanceOf[StructObjectInspector]) - work.commandType match { - case (SparkLoadWork.CommandTypes.OVERWRITE | SparkLoadWork.CommandTypes.NEW_ENTRY) => - memoryTable.put(tablePartitionRDD, tableStats.toMap) - case SparkLoadWork.CommandTypes.INSERT => { - memoryTable.update(tablePartitionRDD, tableStats) + serDe.getObjectInspector.asInstanceOf[StructObjectInspector], + MemoryMetadataManager.makeTableKey(databaseName, tableName), + hivePartitionKeyOpt = None) + if (work.cacheMode != CacheType.TACHYON) { + val memoryTable = getOrCreateMemoryTable(hiveTable) + work.commandType match { + case (SparkLoadWork.CommandTypes.OVERWRITE | SparkLoadWork.CommandTypes.NEW_ENTRY) => + memoryTable.put(tablePartitionRDD, tableStats.toMap) + case SparkLoadWork.CommandTypes.INSERT => { + memoryTable.update(tablePartitionRDD, tableStats) + } } } } @@ -336,7 +374,6 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe for (partSpec <- partSpecs) { // Read, materialize, and store a columnar-backed RDD for `partSpec`. - val partitionedTable = getOrCreatePartitionedMemoryTable(hiveTable, partSpec) val partitionKey = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) val partition = db.getPartition(hiveTable, partSpec, false /* forceCreate */) val partSerDe = partition.getDeserializer() @@ -352,13 +389,18 @@ class SparkLoadTask extends HiveTask[SparkLoadWork] with Serializable with LogHe inputRDD, SparkLoadTask.addPartitionInfoToSerDeProps(partCols, partition.getSchema), hadoopReader.broadcastedHiveConf, - unionOI) - // Determine how to cache the table RDD created. - if (partitionedTable.containsPartition(partitionKey) && - (work.commandType == SparkLoadWork.CommandTypes.INSERT)) { - partitionedTable.updatePartition(partitionKey, tablePartitionRDD, tableStats) - } else { - partitionedTable.putPartition(partitionKey, tablePartitionRDD, tableStats.toMap) + unionOI, + MemoryMetadataManager.makeTableKey(databaseName, tableName), + Some(partitionKey)) + if (work.cacheMode != CacheType.TACHYON) { + // Handle appends or overwrites. + val partitionedTable = getOrCreatePartitionedMemoryTable(hiveTable, partSpec) + if (partitionedTable.containsPartition(partitionKey) && + (work.commandType == SparkLoadWork.CommandTypes.INSERT)) { + partitionedTable.updatePartition(partitionKey, tablePartitionRDD, tableStats) + } else { + partitionedTable.putPartition(partitionKey, tablePartitionRDD, tableStats.toMap) + } } } } diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 85a6b913..6ea9efb2 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -77,33 +77,78 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends MemoryTableRe hiveTable: HiveTable, pruningFnOpt: Option[PruningFunctionType] ): RDD[_] = { - // Table is in Tachyon. - val tableKey = SharkEnv.makeTachyonTableKey(_databaseName, _tableName) - if (!SharkEnv.tachyonUtil.tableExists(tableKey)) { - throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") - } - val statsByteBuffer = SharkEnv.tachyonUtil.getTableMetadata(tableKey) - val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( - statsByteBuffer.array()) - logInfo("Fetching table " + tableKey + " stats from Tachyon.") - val (tableRdd, tableStats) = (SharkEnv.tachyonUtil.createRDD(tableKey), indexToStats) - if (pruningFnOpt.isDefined) { - val pruningFn = pruningFnOpt.get - pruningFn(tableRdd, tableStats) - } else { - tableRdd - } + val tableKey = MemoryMetadataManager.makeTableKey(_databaseName, _tableName) + makeRDD(tableKey, hivePartitionKeyOpt = None, pruningFnOpt) } override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { - throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") + makeRDDForPartitionedTable(partitions, pruningFnOpt = None) } override def makeRDDForPartitionedTable( partitions: Seq[HivePartition], - pruningFnOpt: Option[PruningFunctionType] - ): RDD[_] = { - throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") + pruningFnOpt: Option[PruningFunctionType]): RDD[_] = { + val tableKey = MemoryMetadataManager.makeTableKey(_databaseName, _tableName) + val hivePartitionRDDs = partitions.map { hivePartition => + val partDesc = Utilities.getPartitionDesc(hivePartition) + // Get partition field info + val partSpec = partDesc.getPartSpec() + val partProps = partDesc.getProperties() + + val partColsDelimited = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + // Partitioning columns are delimited by "/" + val partCols = partColsDelimited.trim().split("/").toSeq + // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. + val partValues = if (partSpec == null) { + Array.fill(partCols.size)(new String) + } else { + partCols.map(col => new String(partSpec.get(col))).toArray + } + val partitionKeyStr = MemoryMetadataManager.makeHivePartitionKeyStr(partCols, partSpec) + val hivePartitionRDD = makeRDD(tableKey, Some(partitionKeyStr), pruningFnOpt) + hivePartitionRDD.mapPartitions { iter => + if (iter.hasNext) { + // Map each tuple to a row object + val rowWithPartArr = new Array[Object](2) + iter.map { value => + rowWithPartArr.update(0, value.asInstanceOf[Object]) + rowWithPartArr.update(1, partValues) + rowWithPartArr.asInstanceOf[Object] + } + } else { + Iterator.empty + } + } + } + if (hivePartitionRDDs.size > 0) { + new UnionRDD(hivePartitionRDDs.head.context, hivePartitionRDDs) + } else { + new EmptyRDD[Object](SharkEnv.sc) + } + } + + private def makeRDD( + tableKey: String, + hivePartitionKeyOpt: Option[String], + pruningFnOpt: Option[PruningFunctionType]): RDD[Any] = { + // Check that the table is in Tachyon. + if (!SharkEnv.tachyonUtil.tableExists(tableKey, hivePartitionKeyOpt)) { + throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") + } + val tableRDDsAndStats = SharkEnv.tachyonUtil.createRDD(tableKey, hivePartitionKeyOpt) + val prunedRDDs = if (pruningFnOpt.isDefined) { + val pruningFn = pruningFnOpt.get + tableRDDsAndStats.map(tableRDDWithStats => + pruningFn(tableRDDWithStats._1, tableRDDWithStats._2).asInstanceOf[RDD[Any]]) + } else { + tableRDDsAndStats.map(tableRDDAndStats => tableRDDAndStats._1.asInstanceOf[RDD[Any]]) + } + val unionedRDD = if (prunedRDDs.isEmpty) { + new EmptyRDD[TablePartition](SharkEnv.sc) + } else { + new UnionRDD(SharkEnv.sc, prunedRDDs) + } + unionedRDD.asInstanceOf[RDD[Any]] } } diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index 74c3161d..c5684ca2 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -82,8 +82,10 @@ class TableScanOperator extends TopOperator[TableScanDesc] { } override def outputObjectInspector() = { + val cacheMode = CacheType.fromString( + tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) if (parts == null) { - val serializer = if (isInMemoryTableScan) { + val serializer = if (CacheType.shouldCache(cacheMode)) { new ColumnarSerDe } else { tableDesc.getDeserializerClass().newInstance() @@ -92,7 +94,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { serializer.getObjectInspector() } else { val partProps = firstConfPartDesc.getProperties() - val partSerDe = if (isInMemoryTableScan) { + val partSerDe = if (CacheType.shouldCache(cacheMode)) { new ColumnarSerDe } else { firstConfPartDesc.getDeserializerClass().newInstance() @@ -116,12 +118,12 @@ class TableScanOperator extends TopOperator[TableScanDesc] { val cacheMode = CacheType.fromString( tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. - if (isInMemoryTableScan) { - assert ( - cacheMode == CacheType.MEMORY || cacheMode == CacheType.MEMORY_ONLY || - cacheMode == CacheType.TACHYON, - "Table %s.%s is in Shark metastore, but its cacheMode (%s) indicates otherwise". - format(databaseName, tableName, cacheMode)) + if (isInMemoryTableScan || cacheMode == CacheType.TACHYON) { + if (isInMemoryTableScan) { + assert(cacheMode == CacheType.MEMORY || cacheMode == CacheType.MEMORY_ONLY, + "Table %s.%s is in Shark metastore, but its cacheMode (%s) indicates otherwise". + format(databaseName, tableName, cacheMode)) + } val tableReader = if (cacheMode == CacheType.TACHYON) { new TachyonTableReader(tableDesc) } else { @@ -144,7 +146,12 @@ class TableScanOperator extends TopOperator[TableScanDesc] { // Run map pruning if the flag is set, there exists a filter predicate on // the input table and we have statistics on the table. val columnsUsed = new ColumnPruner(this, table).columnsUsed - SharkEnv.tachyonUtil.pushDownColumnPruning(rdd, columnsUsed) + + val cacheMode = CacheType.fromString( + tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) + if (!table.isPartitioned && cacheMode == CacheType.TACHYON) { + SharkEnv.tachyonUtil.pushDownColumnPruning(rdd, columnsUsed) + } val shouldPrune = SharkConfVars.getBoolVar(localHConf, SharkConfVars.MAP_PRUNING) && childOperators(0).isInstanceOf[FilterOperator] && @@ -183,7 +190,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { rdd } - return prunedRdd.mapPartitions { iter => + prunedRdd.mapPartitions { iter => if (iter.hasNext) { val tablePartition1 = iter.next() val tablePartition = tablePartition1.asInstanceOf[TablePartition] diff --git a/src/main/scala/shark/memstore2/LazySimpleSerDeWrapper.scala b/src/main/scala/shark/memstore2/LazySimpleSerDeWrapper.scala new file mode 100644 index 00000000..2211d557 --- /dev/null +++ b/src/main/scala/shark/memstore2/LazySimpleSerDeWrapper.scala @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.util.{List => JList, Properties} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.serde2.{SerDe, SerDeStats} +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.io.Writable + + +class LazySimpleSerDeWrapper extends SerDe { + + val _lazySimpleSerDe = new LazySimpleSerDe() + + override def initialize(conf: Configuration, tbl: Properties) { + _lazySimpleSerDe.initialize(conf, tbl) + } + + override def deserialize(blob: Writable): Object = _lazySimpleSerDe.deserialize(blob) + + override def getSerDeStats(): SerDeStats = _lazySimpleSerDe.getSerDeStats() + + override def getObjectInspector: ObjectInspector = _lazySimpleSerDe.getObjectInspector + + override def getSerializedClass: Class[_ <: Writable] = _lazySimpleSerDe.getSerializedClass + + override def serialize(obj: Object, objInspector: ObjectInspector): Writable = { + _lazySimpleSerDe.serialize(obj, objInspector) + } + +} diff --git a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala index 74f483ad..9d5ce7ab 100755 --- a/src/main/scala/shark/memstore2/MemoryMetadataManager.scala +++ b/src/main/scala/shark/memstore2/MemoryMetadataManager.scala @@ -17,8 +17,8 @@ package shark.memstore2 -import java.util.concurrent.ConcurrentHashMap import java.util.{HashMap=> JavaHashMap, Map => JavaMap} +import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions._ import scala.collection.concurrent @@ -27,18 +27,19 @@ import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.spark.rdd.{RDD, UnionRDD} +import shark.{LogHelper, SharkEnv} import shark.execution.RDDUtils import shark.util.HiveUtils -class MemoryMetadataManager { +class MemoryMetadataManager extends LogHelper { // Set of tables, from databaseName.tableName to Table object. private val _tables: concurrent.Map[String, Table] = new ConcurrentHashMap[String, Table]() def isHivePartitioned(databaseName: String, tableName: String): Boolean = { - val tableKey = makeTableKey(databaseName, tableName) + val tableKey = MemoryMetadataManager.makeTableKey(databaseName, tableName) _tables.get(tableKey) match { case Some(table) => table.isInstanceOf[PartitionedMemoryTable] case None => false @@ -46,14 +47,14 @@ class MemoryMetadataManager { } def containsTable(databaseName: String, tableName: String): Boolean = { - _tables.contains(makeTableKey(databaseName, tableName)) + _tables.contains(MemoryMetadataManager.makeTableKey(databaseName, tableName)) } def createMemoryTable( databaseName: String, tableName: String, cacheMode: CacheType.CacheType): MemoryTable = { - val tableKey = makeTableKey(databaseName, tableName) + val tableKey = MemoryMetadataManager.makeTableKey(databaseName, tableName) val newTable = new MemoryTable(databaseName, tableName, cacheMode) _tables.put(tableKey, newTable) newTable @@ -65,7 +66,7 @@ class MemoryMetadataManager { cacheMode: CacheType.CacheType, tblProps: JavaMap[String, String] ): PartitionedMemoryTable = { - val tableKey = makeTableKey(databaseName, tableName) + val tableKey = MemoryMetadataManager.makeTableKey(databaseName, tableName) val newTable = new PartitionedMemoryTable(databaseName, tableName, cacheMode) // Determine the cache policy to use and read any user-specified cache settings. val cachePolicyStr = tblProps.getOrElse(SharkTblProperties.CACHE_POLICY.varname, @@ -79,11 +80,11 @@ class MemoryMetadataManager { } def getTable(databaseName: String, tableName: String): Option[Table] = { - _tables.get(makeTableKey(databaseName, tableName)) + _tables.get(MemoryMetadataManager.makeTableKey(databaseName, tableName)) } def getMemoryTable(databaseName: String, tableName: String): Option[MemoryTable] = { - val tableKey = makeTableKey(databaseName, tableName) + val tableKey = MemoryMetadataManager.makeTableKey(databaseName, tableName) val tableOpt = _tables.get(tableKey) if (tableOpt.isDefined) { assert(tableOpt.get.isInstanceOf[MemoryTable], @@ -95,7 +96,7 @@ class MemoryMetadataManager { def getPartitionedTable( databaseName: String, tableName: String): Option[PartitionedMemoryTable] = { - val tableKey = makeTableKey(databaseName, tableName) + val tableKey = MemoryMetadataManager.makeTableKey(databaseName, tableName) val tableOpt = _tables.get(tableKey) if (tableOpt.isDefined) { assert(tableOpt.get.isInstanceOf[PartitionedMemoryTable], @@ -106,8 +107,8 @@ class MemoryMetadataManager { def renameTable(databaseName: String, oldName: String, newName: String) { if (containsTable(databaseName, oldName)) { - val oldTableKey = makeTableKey(databaseName, oldName) - val newTableKey = makeTableKey(databaseName, newName) + val oldTableKey = MemoryMetadataManager.makeTableKey(databaseName, oldName) + val newTableKey = MemoryMetadataManager.makeTableKey(databaseName, newName) val tableValueEntry = _tables.remove(oldTableKey).get tableValueEntry.tableName = newTableKey @@ -117,19 +118,22 @@ class MemoryMetadataManager { } /** - * Used to drop a table from the Spark in-memory cache and/or disk. All metadata tracked by Shark - * is deleted as well. + * Used to drop a table from Spark in-memory cache and/or disk. All metadata is deleted as well. + * + * Note that this is always used in conjunction with a dropTableFromMemory() for handling + *'shark.cache' property changes in an ALTER TABLE command, or to finish off a DROP TABLE command + * after the table has been deleted from the Hive metastore. * * @return Option::isEmpty() is true of there is no MemoryTable (and RDD) corresponding to 'key' - * in _keyToMemoryTable. For MemoryTables that are Hive-partitioned, the RDD returned will - * be a UnionRDD comprising RDDs that represent the table's Hive-partitions. + * in _keyToMemoryTable. For tables that are Hive-partitioned, the RDD returned will be a + * UnionRDD comprising RDDs that back the table's Hive-partitions. */ - def removeTable(databaseName: String, tableName: String): Option[RDD[_]] = { - val tableKey = makeTableKey(databaseName, tableName) - - // Remove MemoryTable's entry from Shark metadata. - val tableValue: Option[Table] = _tables.remove(tableKey) - tableValue.flatMap(MemoryMetadataManager.unpersistRDDsInTable(_)) + def removeTable( + databaseName: String, + tableName: String): Option[RDD[_]] = { + val tableKey = MemoryMetadataManager.makeTableKey(databaseName, tableName) + val tableValueOpt: Option[Table] = _tables.remove(tableKey) + tableValueOpt.flatMap(tableValue => MemoryMetadataManager.unpersistRDDsForTable(tableValue)) } def shutdown() { @@ -167,31 +171,33 @@ class MemoryMetadataManager { removeTable(databaseName, tableName) } } - - // Returns the key "databaseName.tableName". - private def makeTableKey(databaseName: String, tableName: String): String = { - (databaseName + '.' + tableName).toLowerCase - } } object MemoryMetadataManager { - def unpersistRDDsInTable(table: Table): Option[RDD[_]] = { - var unpersistedRDD: Option[RDD[_]] = None - if (table.isInstanceOf[PartitionedMemoryTable]) { - val partitionedTable = table.asInstanceOf[PartitionedMemoryTable] - // unpersist() all RDDs for all Hive-partitions. - val unpersistedRDDs = partitionedTable.keyToPartitions.values.map( - rdd => RDDUtils.unpersistRDD(rdd)).asInstanceOf[Seq[RDD[Any]]] - if (unpersistedRDDs.size > 0) { - val unionedRDD = new UnionRDD(unpersistedRDDs.head.context, unpersistedRDDs) - unpersistedRDD = Some(unionedRDD) + def unpersistRDDsForTable(table: Table): Option[RDD[_]] = { + table match { + case partitionedTable: PartitionedMemoryTable => { + // unpersist() all RDDs for all Hive-partitions. + val unpersistedRDDs = partitionedTable.keyToPartitions.values.map(rdd => + RDDUtils.unpersistRDD(rdd)).asInstanceOf[Seq[RDD[Any]]] + if (unpersistedRDDs.size > 0) { + val unionedRDD = new UnionRDD(unpersistedRDDs.head.context, unpersistedRDDs) + Some(unionedRDD) + } else { + None + } } - } else { - unpersistedRDD = Some(RDDUtils.unpersistRDD(table.asInstanceOf[MemoryTable].getRDD.get)) + case memoryTable: MemoryTable => Some(RDDUtils.unpersistRDD(memoryTable.getRDD.get)) } - unpersistedRDD + } + + // Returns a key of the form "databaseName.tableName" that uniquely identifies a Shark table. + // For example, it's used to track a table's RDDs in MemoryMetadataManager and table paths in the + // Tachyon table warehouse. + def makeTableKey(databaseName: String, tableName: String): String = { + (databaseName + '.' + tableName).toLowerCase } /** diff --git a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala index 820bf5f0..b6bd8ae6 100644 --- a/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala +++ b/src/main/scala/shark/memstore2/PartitionedMemoryTable.scala @@ -123,11 +123,18 @@ class PartitionedMemoryTable( rddRemoved.map(_.toTuple) } + /** Returns an immutable view of (partition key -> RDD) mappings to external callers */ + def keyToPartitions: collection.immutable.Map[String, RDD[TablePartition]] = { + _keyToPartitions.mapValues(_.rdd).toMap + } + def setPartitionCachePolicy(cachePolicyStr: String, fallbackMaxSize: Int) { // The loadFunc will upgrade the persistence level of the RDD to the preferred storage level. val loadFunc: String => RDDValue = (partitionKey: String) => { val rddValue = _keyToPartitions.get(partitionKey).get - rddValue.rdd.persist(StorageLevel.MEMORY_AND_DISK) + if (cacheMode == CacheType.MEMORY) { + rddValue.rdd.persist(StorageLevel.MEMORY_AND_DISK) + } rddValue } // The evictionFunc will unpersist the RDD. @@ -141,9 +148,4 @@ class PartitionedMemoryTable( def cachePolicy: CachePolicy[String, RDDValue] = _cachePolicy - /** Returns an immutable view of (partition key -> RDD) mappings to external callers */ - def keyToPartitions: collection.immutable.Map[String, RDD[TablePartition]] = { - _keyToPartitions.mapValues(_.rdd).toMap - } - } diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala index d671aa8d..21fbddb7 100644 --- a/src/main/scala/shark/memstore2/TableRecovery.scala +++ b/src/main/scala/shark/memstore2/TableRecovery.scala @@ -17,11 +17,13 @@ package shark.memstore2 +import java.util.{HashMap => JavaHashMap} + import scala.collection.JavaConversions.asScalaBuffer import org.apache.hadoop.hive.ql.metadata.Hive -import shark.LogHelper +import shark.{LogHelper, SharkEnv} import shark.util.QueryRewriteUtils /** @@ -42,7 +44,8 @@ object TableRecovery extends LogHelper { // Filter for tables that should be reloaded into the cache. val currentDbName = db.getCurrentDatabase() for (databaseName <- db.getAllDatabases(); tableName <- db.getAllTables(databaseName)) { - val tblProps = db.getTable(databaseName, tableName).getParameters + val hiveTable = db.getTable(databaseName, tableName) + val tblProps = hiveTable.getParameters val cacheMode = CacheType.fromString(tblProps.get(SharkTblProperties.CACHE_FLAG.varname)) if (cacheMode == CacheType.MEMORY) { logInfo("Reloading %s.%s into memory.".format(databaseName, tableName)) diff --git a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala index f0a884ca..3e5f69b2 100644 --- a/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkDDLSemanticAnalyzer.scala @@ -83,8 +83,12 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) val oldCacheMode = CacheType.fromString(oldTblProps.get(SharkTblProperties.CACHE_FLAG.varname)) val newCacheMode = CacheType.fromString(newTblProps.get(SharkTblProperties.CACHE_FLAG.varname)) - val isAlreadyCached = SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName) - if (!isAlreadyCached && newCacheMode == CacheType.MEMORY) { + if ((oldCacheMode == CacheType.TACHYON && newCacheMode != CacheType.TACHYON) || + (oldCacheMode == CacheType.MEMORY_ONLY && newCacheMode != CacheType.MEMORY_ONLY)) { + throw new SemanticException("""Table %s.%s's 'shark.cache' table property is %s. Only changes + from "'MEMORY' and 'NONE' are supported. Tables stored in TACHYON and MEMORY_ONLY must be + "dropped.""".format(databaseName, tableName, oldCacheMode)) + } else if (newCacheMode == CacheType.MEMORY) { // The table should be cached (and is not already cached). val partSpecsOpt = if (hiveTable.isPartitioned) { val columnNames = hiveTable.getPartCols.map(_.getName) @@ -106,30 +110,30 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) newCacheMode) partSpecsOpt.foreach(partSpecs => sparkLoadWork.partSpecs = partSpecs) rootTasks.head.addDependentTask(TaskFactory.get(sparkLoadWork, conf)) + } else if (newCacheMode == CacheType.NONE) { + // Uncache the table. + SharkEnv.memoryMetadataManager.dropTableFromMemory(db, databaseName, tableName) } - if (CacheType.shouldCache(oldCacheMode) && !CacheType.shouldCache(newCacheMode)) { - if (oldCacheMode == CacheType.MEMORY) { - // Uncache the table. - SharkEnv.memoryMetadataManager.dropTableFromMemory(db, databaseName, tableName) - } else { - throw new SemanticException( - "A memory-only table should be dropped.") - } - } - } def analyzeDropTableOrDropParts(ast: ASTNode) { val databaseName = db.getCurrentDatabase() val tableName = getTableName(ast) - // Create a SharkDDLTask only if the table is cached. - if (SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { - // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks - // and DDLWorks that contain DropTableDesc objects. - for (ddlTask <- rootTasks) { - val dropTableDesc = ddlTask.getWork.asInstanceOf[DDLWork].getDropTblDesc - val sharkDDLWork = new SharkDDLWork(dropTableDesc) - ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) + val hiveTableOpt = Option(db.getTable(databaseName, tableName, false /* throwException */)) + // `hiveTableOpt` can be NONE for a DROP TABLE IF EXISTS command on a nonexistent table. + hiveTableOpt.foreach { hiveTable => + val cacheMode = CacheType.fromString( + hiveTable.getProperty(SharkTblProperties.CACHE_FLAG.varname)) + // Create a SharkDDLTask only if the table is cached. + if (CacheType.shouldCache(cacheMode)) { + // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks + // and DDLWorks that contain DropTableDesc objects. + for (ddlTask <- rootTasks) { + val dropTableDesc = ddlTask.getWork.asInstanceOf[DDLWork].getDropTblDesc + val sharkDDLWork = new SharkDDLWork(dropTableDesc) + sharkDDLWork.cacheMode = cacheMode + ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) + } } } } @@ -137,13 +141,17 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) def analyzeAlterTableAddParts(ast: ASTNode) { val databaseName = db.getCurrentDatabase() val tableName = getTableName(ast) + val hiveTable = db.getTable(databaseName, tableName) + val cacheMode = CacheType.fromString( + hiveTable.getProperty(SharkTblProperties.CACHE_FLAG.varname)) // Create a SharkDDLTask only if the table is cached. - if (SharkEnv.memoryMetadataManager.containsTable(databaseName, tableName)) { + if (CacheType.shouldCache(cacheMode)) { // Hive's DDLSemanticAnalyzer#analyzeInternal() will only populate rootTasks with DDLTasks // and DDLWorks that contain AddPartitionDesc objects. for (ddlTask <- rootTasks) { val addPartitionDesc = ddlTask.getWork.asInstanceOf[DDLWork].getAddPartitionDesc val sharkDDLWork = new SharkDDLWork(addPartitionDesc) + sharkDDLWork.cacheMode = cacheMode ddlTask.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } @@ -152,11 +160,12 @@ class SharkDDLSemanticAnalyzer(conf: HiveConf) extends DDLSemanticAnalyzer(conf) private def analyzeAlterTableRename(astNode: ASTNode) { val databaseName = db.getCurrentDatabase() val oldTableName = getTableName(astNode) - if (SharkEnv.memoryMetadataManager.containsTable(databaseName, oldTableName)) { - val newTableName = BaseSemanticAnalyzer.getUnescapedName( - astNode.getChild(1).asInstanceOf[ASTNode]) + val hiveTable = db.getTable(databaseName, oldTableName) + val cacheMode = CacheType.fromString(hiveTable.getProperty(SharkTblProperties.CACHE_FLAG.varname)) + if (CacheType.shouldCache(cacheMode)) { val alterTableDesc = getAlterTblDesc() val sharkDDLWork = new SharkDDLWork(alterTableDesc) + sharkDDLWork.cacheMode = cacheMode rootTasks.head.addDependentTask(TaskFactory.get(sharkDDLWork, conf)) } } diff --git a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala index f7a186b0..fc32dbd7 100644 --- a/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkLoadSemanticAnalyzer.scala @@ -27,7 +27,8 @@ import org.apache.hadoop.hive.ql.plan._ import shark.{LogHelper, SharkEnv} import shark.execution.SparkLoadWork -import shark.memstore2.CacheType +import shark.memstore2.{CacheType, SharkTblProperties} + class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(conf) { @@ -41,10 +42,11 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con // 3. (optional) node representing the LOCAL modifier. val tableASTNode = ast.getChild(1).asInstanceOf[ASTNode] val tableName = getTableName(tableASTNode) - val databaseName = db.getCurrentDatabase() + val hiveTable = db.getTable(tableName) + val cacheMode = CacheType.fromString( + hiveTable.getProperty(SharkTblProperties.CACHE_FLAG.varname)) - val tableOpt = SharkEnv.memoryMetadataManager.getTable(databaseName, tableName) - if (tableOpt.exists(table => table.cacheMode == CacheType.MEMORY)) { + if (CacheType.shouldCache(cacheMode)) { // Find the arguments needed to instantiate a SparkLoadWork. val tableSpec = new BaseSemanticAnalyzer.tableSpec(db, conf, tableASTNode) val hiveTable = tableSpec.tableHandle @@ -85,4 +87,3 @@ class SharkLoadSemanticAnalyzer(conf: HiveConf) extends LoadSemanticAnalyzer(con BaseSemanticAnalyzer.getUnescapedName(node.getChild(0).asInstanceOf[ASTNode]) } } - diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index cdbe8ba3..a995f77d 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -27,12 +27,13 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.Warehouse import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.ql.exec.{DDLTask, FetchTask} import org.apache.hadoop.hive.ql.exec.{FileSinkOperator => HiveFileSinkOperator} import org.apache.hadoop.hive.ql.exec.MoveTask import org.apache.hadoop.hive.ql.exec.{Operator => HiveOperator} import org.apache.hadoop.hive.ql.exec.TaskFactory -import org.apache.hadoop.hive.ql.metadata.HiveException +import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException} import org.apache.hadoop.hive.ql.optimizer.Optimizer import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan._ @@ -41,7 +42,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import shark.{LogHelper, SharkConfVars, SharkEnv, Utils} import shark.execution.{HiveDesc, Operator, OperatorFactory, RDDUtils, ReduceSinkOperator} import shark.execution.{SharkDDLWork, SparkLoadWork, SparkWork, TerminalOperator} -import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} +import shark.memstore2.{CacheType, ColumnarSerDe, LazySimpleSerDeWrapper, MemoryMetadataManager} import shark.memstore2.{MemoryTable, PartitionedMemoryTable, SharkTblProperties, TableRecovery} @@ -182,30 +183,26 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with val tableNameSplit = tableName.split('.') // Split from 'databaseName.tableName' val cachedTableName = tableNameSplit(1) val databaseName = tableNameSplit(0) - if (SharkEnv.memoryMetadataManager.containsTable(databaseName, cachedTableName)) { + val hiveTable = Hive.get().getTable(databaseName, tableName) + val cacheMode = CacheType.fromString( + hiveTable.getProperty(SharkTblProperties.CACHE_FLAG.varname)) + if (CacheType.shouldCache(cacheMode)) { if (hiveSinkOps.size == 1) { - val table = SharkEnv.memoryMetadataManager.getTable( - databaseName, cachedTableName).get // INSERT INTO or OVERWRITE update on a cached table. qb.targetTableDesc = tableDesc // If useUnionRDD is true, the sink op is for INSERT INTO. val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) - val isPartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( - databaseName, cachedTableName) - var hivePartitionKey = if (isPartitioned) { - if (table.cacheMode == CacheType.TACHYON) { - throw new SemanticException( - "Shark does not support caching Hive-partitioned table(s) in Tachyon.") - } - SharkSemanticAnalyzer.getHivePartitionKey(qb) + val isPartitioned = hiveTable.isPartitioned + var hivePartitionKeyOpt = if (isPartitioned) { + Some(SharkSemanticAnalyzer.getHivePartitionKey(qb)) } else { - new String + None } - if (table.cacheMode == CacheType.MEMORY) { + if (cacheMode == CacheType.MEMORY) { // The table being updated is stored in memory and backed by disk, a // SparkLoadTask will be created by the genMapRedTasks() call below. Set fields // in `qb` that will be needed. - qb.cacheMode = table.cacheMode + qb.cacheMode = cacheMode qb.targetTableDesc = tableDesc OperatorFactory.createSharkFileOutputPlan(hiveSinkOp) } else { @@ -214,8 +211,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with cachedTableName, databaseName, _resSchema.size, /* numColumns */ - hivePartitionKey, - table.cacheMode, + hivePartitionKeyOpt, + cacheMode, useUnionRDD) } } else { @@ -243,10 +240,10 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with hiveSinkOps.head, qb.createTableDesc.getTableName, qb.createTableDesc.getDatabaseName, - _resSchema.size, /* numColumns */ - new String, /* hivePartitionKey */ + numColumns = _resSchema.size, /* numColumns */ + hivePartitionKeyOpt = None, qb.cacheMode, - false /* useUnionRDD */) + useUnionRDD = false) } } else if (pctx.getContext().asInstanceOf[QueryContext].useTableRddSink && !qb.isCTAS) { OperatorFactory.createSharkRddOutputPlan(hiveSinkOps.head) @@ -483,9 +480,14 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with // Continue planning based on the 'cacheMode' read. val shouldCache = CacheType.shouldCache(cacheMode) if (shouldCache) { - if (cacheMode == CacheType.MEMORY_ONLY) { - // Directly set the ColumnarSerDe property. - createTableDesc.setSerName(classOf[ColumnarSerDe].getName) + if (cacheMode == CacheType.MEMORY_ONLY || cacheMode == CacheType.TACHYON) { + val serDeName = createTableDesc.getSerName + if (serDeName == null || serDeName == classOf[LazySimpleSerDe].getName) { + // Hive's SemanticAnalyzer optimizes based on checks for LazySimpleSerDe, which causes + // casting exceptions for cached table scans during runtime. Use a simple SerDe wrapper + // to guard against these optimizations. + createTableDesc.setSerName(classOf[LazySimpleSerDeWrapper].getName) + } } createTableProperties.put(SharkTblProperties.CACHE_FLAG.varname, cacheMode.toString) } diff --git a/src/main/scala/shark/tachyon/TachyonUtil.scala b/src/main/scala/shark/tachyon/TachyonUtil.scala index 3a50eead..25207d91 100644 --- a/src/main/scala/shark/tachyon/TachyonUtil.scala +++ b/src/main/scala/shark/tachyon/TachyonUtil.scala @@ -22,8 +22,7 @@ import java.util.BitSet import org.apache.spark.rdd.RDD -import shark.memstore2.TablePartition - +import shark.memstore2.{TablePartition, TablePartitionStats} /** @@ -32,17 +31,27 @@ import shark.memstore2.TablePartition * even without Tachyon jars. */ abstract class TachyonUtil { + def pushDownColumnPruning(rdd: RDD[_], columnUsed: BitSet): Boolean def tachyonEnabled(): Boolean - def tableExists(tableName: String): Boolean + def tableExists(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean + + def dropTable(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean - def dropTable(tableName: String): Boolean + def createDirectory(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean - def getTableMetadata(tableName: String): ByteBuffer + def renameDirectory(oldName: String, newName: String): Boolean - def createRDD(tableName: String): RDD[TablePartition] + def createRDD( + tableKey: String, + hivePartitionKeyOpt: Option[String] + ): Seq[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] - def createTableWriter(tableName: String, numColumns: Int): TachyonTableWriter + def createTableWriter( + tableKey: String, + hivePartitionKey: Option[String], + numColumns: Int + ): TachyonTableWriter } diff --git a/src/tachyon_disabled/scala/shark/tachyon/TachyonUtilImpl.scala b/src/tachyon_disabled/scala/shark/tachyon/TachyonUtilImpl.scala index 3f1d2eba..dbdf1ff6 100644 --- a/src/tachyon_disabled/scala/shark/tachyon/TachyonUtilImpl.scala +++ b/src/tachyon_disabled/scala/shark/tachyon/TachyonUtilImpl.scala @@ -22,35 +22,51 @@ import java.util.BitSet import org.apache.spark.rdd.RDD -import shark.memstore2.TablePartition - +import shark.memstore2.{Table, TablePartition, TablePartitionStats} class TachyonUtilImpl(val master: String, val warehousePath: String) extends TachyonUtil { + override def pushDownColumnPruning(rdd: RDD[_], columnUsed: BitSet): Boolean = false override def tachyonEnabled(): Boolean = false - override def tableExists(tableName: String): Boolean = { + override def tableExists(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean = { + throw new UnsupportedOperationException( + "This version of Shark is not compiled with Tachyon support.") + } + + override def dropTable(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean = { throw new UnsupportedOperationException( "This version of Shark is not compiled with Tachyon support.") } - override def dropTable(tableName: String): Boolean = { + override def createDirectory( + tableKey: String, + hivePartitionKeyOpt: Option[String]): Boolean = { throw new UnsupportedOperationException( "This version of Shark is not compiled with Tachyon support.") } - override def getTableMetadata(tableName: String): ByteBuffer = { + override def renameDirectory( + oldName: String, + newName: String): Boolean = { throw new UnsupportedOperationException( "This version of Shark is not compiled with Tachyon support.") } - override def createRDD(tableName: String): RDD[TablePartition] = { + override def createRDD( + tableKey: String, + hivePartitionKeyOpt: Option[String] + ): Seq[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { throw new UnsupportedOperationException( "This version of Shark is not compiled with Tachyon support.") } - override def createTableWriter(tableName: String, numColumns: Int): TachyonTableWriter = { + override def createTableWriter( + tableKey: String, + hivePartitionKeyOpt: Option[String], + numColumns: Int + ): TachyonTableWriter = { throw new UnsupportedOperationException( "This version of Shark is not compiled with Tachyon support.") } diff --git a/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala b/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala index 32f27dee..49b64c4e 100644 --- a/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala +++ b/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala @@ -19,67 +19,127 @@ package shark.tachyon import java.nio.ByteBuffer import java.util.BitSet +import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} -import scala.collection.JavaConverters._ +import scala.collection.JavaConversions._ -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{EmptyRDD, RDD, UnionRDD} import tachyon.client.TachyonFS import tachyon.client.table.{RawTable, RawColumn} -import shark.SharkEnv -import shark.memstore2.TablePartition +import shark.{LogHelper, SharkEnv} +import shark.execution.serialization.JavaSerializer +import shark.memstore2.{MemoryMetadataManager, TablePartition, TablePartitionStats} /** * An abstraction for the Tachyon APIs. */ -class TachyonUtilImpl(val master: String, val warehousePath: String) extends TachyonUtil { +class TachyonUtilImpl( + val master: String, + val warehousePath: String) + extends TachyonUtil + with LogHelper { + + private val INSERT_FILE_PREFIX = "insert_" + + private val _fileNameMappings = new ConcurrentJavaHashMap[String, Int]() val client = if (master != null && master != "") TachyonFS.get(master) else null + private def getUniqueFilePath(parentDirectory: String): String = { + val parentDirectoryLower = parentDirectory.toLowerCase + val currentInsertNum = if (_fileNameMappings.containsKey(parentDirectoryLower)) { + _fileNameMappings.get(parentDirectoryLower) + } else { + 0 + } + var nextInsertNum = currentInsertNum + 1 + var filePath = parentDirectoryLower + "/" + INSERT_FILE_PREFIX + // Make sure there aren't file conflicts. This could occur if the directory was created in a + // previous Shark session. + while (client.exist(filePath + nextInsertNum)) { + nextInsertNum = nextInsertNum + 1 + } + _fileNameMappings.put(parentDirectoryLower, nextInsertNum) + filePath + nextInsertNum + } + if (master != null && warehousePath == null) { throw new TachyonException("TACHYON_MASTER is set. However, TACHYON_WAREHOUSE_PATH is not.") } - def getPath(tableName: String): String = warehousePath + "/" + tableName + private def getPath(tableKey: String, hivePartitionKeyOpt: Option[String]): String = { + val hivePartitionKey = if (hivePartitionKeyOpt.isDefined) { + "/" + hivePartitionKeyOpt.get + } else { + "" + } + warehousePath + "/" + tableKey + hivePartitionKey + } override def pushDownColumnPruning(rdd: RDD[_], columnUsed: BitSet): Boolean = { - if (rdd.isInstanceOf[TachyonTableRDD]) { + val isTachyonTableRdd = rdd.isInstanceOf[TachyonTableRDD] + if (isTachyonTableRdd) { rdd.asInstanceOf[TachyonTableRDD].setColumnUsed(columnUsed) - true - } else { - false } + isTachyonTableRdd } - override def tachyonEnabled(): Boolean = (master != null && warehousePath != null) - override def tableExists(tableName: String): Boolean = { - client.exist(getPath(tableName)) + override def tableExists(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean = { + client.exist(getPath(tableKey, hivePartitionKeyOpt)) } - override def dropTable(tableName: String): Boolean = { + override def dropTable(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean = { // The second parameter (true) means recursive deletion. - client.delete(getPath(tableName), true) + client.delete(getPath(tableKey, hivePartitionKeyOpt), true) } - override def getTableMetadata(tableName: String): ByteBuffer = { - if (!tableExists(tableName)) { - throw new TachyonException("Table " + tableName + " does not exist in Tachyon") - } - client.getRawTable(getPath(tableName)).getMetadata() + override def createDirectory( + tableKey: String, + hivePartitionKeyOpt: Option[String]): Boolean = { + client.mkdir(getPath(tableKey, hivePartitionKeyOpt)) + } + + override def renameDirectory( + oldTableKey: String, + newTableKey: String): Boolean = { + val oldPath = getPath(oldTableKey, hivePartitionKeyOpt = None) + val newPath = getPath(newTableKey, hivePartitionKeyOpt = None) + client.rename(oldPath, newPath) } - override def createRDD(tableName: String): RDD[TablePartition] = { - new TachyonTableRDD(getPath(tableName), SharkEnv.sc) + override def createRDD( + tableKey: String, + hivePartitionKeyOpt: Option[String] + ): Seq[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { + // Create a TachyonTableRDD for each raw table file in the directory. + val tableDirectory = getPath(tableKey, hivePartitionKeyOpt) + val files = client.ls(tableDirectory, false /* recursive */) + // The first path is just "{tableDirectory}/", so ignore it. + val rawTableFiles = files.subList(1, files.size) + val tableRDDsAndStats = rawTableFiles.map { filePath => + val serializedMetadata2 = client.getRawTable(filePath).getMetadata + val serializedMetadata = client.getRawTable(client.getFileId(filePath)).getMetadata + val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( + serializedMetadata.array()) + (new TachyonTableRDD(filePath, SharkEnv.sc), indexToStats) + } + tableRDDsAndStats } - override def createTableWriter(tableName: String, numColumns: Int): TachyonTableWriter = { + override def createTableWriter( + tableKey: String, + hivePartitionKeyOpt: Option[String], + numColumns: Int): TachyonTableWriter = { if (!client.exist(warehousePath)) { client.mkdir(warehousePath) } - new TachyonTableWriterImpl(getPath(tableName), numColumns) + val parentDirectory = getPath(tableKey, hivePartitionKeyOpt) + val filePath = getUniqueFilePath(parentDirectory) + new TachyonTableWriterImpl(filePath, numColumns) } } From b08a06cea605fc4d09d1666a90f48726eb00b430 Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 14 Jan 2014 13:06:19 -0800 Subject: [PATCH 310/331] Test suite for Tachyon support --- src/test/scala/shark/SQLSuite.scala | 6 +- src/test/scala/shark/TachyonSQLSuite.scala | 431 +++++++++++++++++++++ 2 files changed, 433 insertions(+), 4 deletions(-) create mode 100644 src/test/scala/shark/TachyonSQLSuite.scala diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index 222057b3..9a8c3ab5 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -30,6 +30,8 @@ import org.apache.spark.storage.StorageLevel import shark.api.QueryExecutionException import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} +// import expectSql() shortcut methods +import shark.SharkRunner._ class SQLSuite extends FunSuite { @@ -102,9 +104,6 @@ class SQLSuite extends FunSuite { assert(diskSum == cacheSum, "Sum of keys from cached and disk contents differ") } - // import expectSql() shortcut methods - import shark.SharkRunner._ - ////////////////////////////////////////////////////////////////////////////// // basic SQL ////////////////////////////////////////////////////////////////////////////// @@ -1034,7 +1033,6 @@ class SQLSuite extends FunSuite { sharkMetastore.shutdown() for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - // Check that the number of rows from the table on disk remains the same. val onDiskCount = sc.sql("select count(*) from %s".format(tableName))(0) val cachedCount = cachedTableCounts(i) diff --git a/src/test/scala/shark/TachyonSQLSuite.scala b/src/test/scala/shark/TachyonSQLSuite.scala new file mode 100644 index 00000000..4593516b --- /dev/null +++ b/src/test/scala/shark/TachyonSQLSuite.scala @@ -0,0 +1,431 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import java.util.{HashMap => JavaHashMap} + +import scala.collection.JavaConversions._ + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.storage.StorageLevel + +import shark.api.QueryExecutionException +import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} +// import expectSql() shortcut methods +import shark.SharkRunner._ + + +class TachyonSQLSuite extends FunSuite with BeforeAndAfterAll { + + val DEFAULT_DB_NAME = DEFAULT_DATABASE_NAME + val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" + + var sc: SharkContext = SharkRunner.init() + var sharkMetastore: MemoryMetadataManager = SharkEnv.memoryMetadataManager + + // Determine if Tachyon enabled at runtime. + val isTachyonEnabled = SharkEnv.tachyonUtil.tachyonEnabled() + + sc.runSql("create table test_tachyon as select * from test") + + override def afterAll() { + sc.runSql("drop table test_tachyon") + } + + private def isTachyonTable( + dbName: String, + tableName: String, + hivePartitionKeyOpt: Option[String] = None): Boolean = { + val tableKey = MemoryMetadataManager.makeTableKey(dbName, tableName) + SharkEnv.tachyonUtil.tableExists(tableKey, hivePartitionKeyOpt) + } + + private def createPartitionedTachyonTable(tableName: String, numPartitionsToCreate: Int) { + sc.runSql("drop table if exists %s".format(tableName)) + sc.runSql(""" + create table %s(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache' = 'tachyon') + """.format(tableName)) + var partitionNum = 1 + while (partitionNum <= numPartitionsToCreate) { + sc.runSql("""insert into table %s partition(keypart = %d) + select * from test_tachyon""".format(tableName, partitionNum)) + partitionNum += 1 + } + assert(isTachyonTable(DEFAULT_DB_NAME, tableName)) + } + + if (isTachyonEnabled) { + + ////////////////////////////////////////////////////////////////////////////// + // basic SQL + ////////////////////////////////////////////////////////////////////////////// + test("count") { + expectSql("select count(*) from test_tachyon", "500") + } + + test("filter") { + expectSql("select * from test_tachyon where key=100 or key=497", + Array("100\tval_100", "100\tval_100", "497\tval_497")) + } + + test("count distinct") { + sc.runSql("set mapred.reduce.tasks=3") + expectSql("select count(distinct key) from test_tachyon", "309") + expectSql( + """|SELECT substr(key,1,1), count(DISTINCT substr(val,5)) from test_tachyon + |GROUP BY substr(key,1,1)""".stripMargin, + Array("0\t1", "1\t71", "2\t69", "3\t62", "4\t74", "5\t6", "6\t5", "7\t6", "8\t8", "9\t7")) + } + + test("count bigint") { + sc.runSql("drop table if exists test_bigint") + sc.runSql("create table test_bigint (key bigint, val string)") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + OVERWRITE INTO TABLE test_bigint""") + sc.runSql("drop table if exists test_bigint_tachyon") + sc.runSql("create table test_bigint_tachyon as select * from test_bigint") + expectSql("select val, count(*) from test_bigint_tachyon where key=484 group by val", + "val_484\t1") + + sc.runSql("drop table if exists test_bigint_tachyon") + } + + test("limit") { + assert(sc.runSql("select * from test_tachyon limit 10").results.length === 10) + assert(sc.runSql("select * from test_tachyon limit 501").results.length === 500) + sc.runSql("drop table if exists test_limit0_tachyon") + assert(sc.runSql("select * from test_tachyon limit 0").results.length === 0) + assert(sc.runSql("create table test_limit0_tachyon as select * from test_tachyon limit 0") + .results.length === 0) + assert(sc.runSql("select * from test_limit0_tachyon limit 0").results.length === 0) + assert(sc.runSql("select * from test_limit0_tachyon limit 1").results.length === 0) + + sc.runSql("drop table if exists test_limit0_tachyon") + } + + ////////////////////////////////////////////////////////////////////////////// + // cache DDL + ////////////////////////////////////////////////////////////////////////////// + test("Use regular CREATE TABLE and '_tachyon' suffix to create Tachyon table") { + sc.runSql("drop table if exists empty_table_tachyon") + sc.runSql("create table empty_table_tachyon(key string, value string)") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_table_tachyon")) + + sc.runSql("drop table if exists empty_table_tachyon") + } + + test("Use regular CREATE TABLE and table properties to create Tachyon table") { + sc.runSql("drop table if exists empty_table_tachyon_tbl_props") + sc.runSql("""create table empty_table_tachyon_tbl_props(key string, value string) + TBLPROPERTIES('shark.cache' = 'tachyon')""") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_table_tachyon_tbl_props")) + + sc.runSql("drop table if exists empty_table_tachyon_tbl_props") + } + + test("Insert into empty Tachyon table") { + sc.runSql("drop table if exists new_table_tachyon") + sc.runSql("create table new_table_tachyon(key string, value string)") + sc.runSql("insert into table new_table_tachyon select * from test where key > -1 limit 499") + expectSql("select count(*) from new_table_tachyon", "499") + + sc.runSql("drop table if exists new_table_tachyon") + } + + test("rename Tachyon table") { + sc.runSql("drop table if exists test_oldname_tachyon") + sc.runSql("drop table if exists test_rename") + sc.runSql("create table test_oldname_tachyon as select * from test") + sc.runSql("alter table test_oldname_tachyon rename to test_rename") + + assert(!isTachyonTable(DEFAULT_DB_NAME, "test_oldname_tachyon")) + assert(isTachyonTable(DEFAULT_DB_NAME, "test_rename")) + + expectSql("select count(*) from test_rename", "500") + + sc.runSql("drop table if exists test_rename") + } + + test("insert into tachyon tables") { + sc.runSql("drop table if exists test1_tachyon") + sc.runSql("create table test1_tachyon as select * from test") + expectSql("select count(*) from test1_tachyon", "500") + sc.runSql("insert into table test1_tachyon select * from test where key > -1 limit 499") + expectSql("select count(*) from test1_tachyon", "999") + + sc.runSql("drop table if exists test1_tachyon") + } + + test("insert overwrite") { + sc.runSql("drop table if exists test2_tachyon") + sc.runSql("create table test2_tachyon as select * from test") + expectSql("select count(*) from test2_tachyon", "500") + sc.runSql("insert overwrite table test2_tachyon select * from test where key > -1 limit 499") + expectSql("select count(*) from test2_tachyon", "499") + + sc.runSql("drop table if exists test2_tachyon") + } + + test("error when attempting to update Tachyon table(s) using command with multiple INSERTs") { + sc.runSql("drop table if exists multi_insert_test") + sc.runSql("drop table if exists multi_insert_test_tachyon") + sc.runSql("create table multi_insert_test as select * from test") + sc.runSql("create table multi_insert_test_tachyon as select * from test") + intercept[QueryExecutionException] { + sc.runSql("""from test + insert into table multi_insert_test select * + insert into table multi_insert_test_tachyon select *""") + } + + sc.runSql("drop table if exists multi_insert_test") + sc.runSql("drop table if exists multi_insert_test_tachyon") + } + + test("create Tachyon table with 'shark.cache' flag in table properties") { + sc.runSql("drop table if exists ctas_tbl_props") + sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='tachyon') as + select * from test""") + assert(isTachyonTable(DEFAULT_DB_NAME, "ctas_tbl_props")) + expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") + + sc.runSql("drop table if exists ctas_tbl_props") + } + + test("tachyon tables with complex types") { + sc.runSql("drop table if exists test_complex_types") + sc.runSql("drop table if exists test_complex_types_tachyon") + sc.runSql("""CREATE TABLE test_complex_types ( + a STRING, b ARRAY, c ARRAY>, d MAP>)""") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/create_nested_type.txt' + overwrite into table test_complex_types""") + sc.runSql("""create table test_complex_types_tachyon TBLPROPERTIES ("shark.cache" = "tachyon") + as select * from test_complex_types""") + + assert(sc.sql("select a from test_complex_types_tachyon where a = 'a0'").head === "a0") + + assert(sc.sql("select b from test_complex_types_tachyon where a = 'a0'").head === + """["b00","b01"]""") + + assert(sc.sql("select c from test_complex_types_tachyon where a = 'a0'").head === + """[{"c001":"C001","c002":"C002"},{"c011":null,"c012":"C012"}]""") + + assert(sc.sql("select d from test_complex_types_tachyon where a = 'a0'").head === + """{"d01":["d011","d012"],"d02":["d021","d022"]}""") + + assert(isTachyonTable(DEFAULT_DB_NAME, "test_complex_types_tachyon")) + + sc.runSql("drop table if exists test_complex_types") + sc.runSql("drop table if exists test_complex_types_tachyon") + } + + test("disable caching in Tachyon by default") { + sc.runSql("set shark.cache.flag.checkTableName=false") + sc.runSql("drop table if exists should_not_be_in_tachyon") + sc.runSql("create table should_not_be_in_tachyon as select * from test") + expectSql("select key from should_not_be_in_tachyon where key = 407", "407") + assert(!isTachyonTable(DEFAULT_DB_NAME, "should_not_be_in_tachyon")) + + sc.runSql("set shark.cache.flag.checkTableName=true") + sc.runSql("drop table if exists should_not_be_in_tachyon") + } + + test("tachyon table name should be case-insensitive") { + sc.runSql("drop table if exists sharkTest5tachyon") + sc.runSql("""create table sharkTest5tachyon TBLPROPERTIES ("shark.cache" = "tachyon") as + select * from test""") + expectSql("select val from sharktest5tachyon where key = 407", "val_407") + assert(isTachyonTable(DEFAULT_DB_NAME, "sharkTest5tachyon")) + + sc.runSql("drop table if exists sharkTest5tachyon") + } + + test("dropping tachyon tables should clean up RDDs") { + sc.runSql("drop table if exists sharkTest5tachyon") + sc.runSql("""create table sharkTest5tachyon TBLPROPERTIES ("shark.cache" = "tachyon") as + select * from test""") + sc.runSql("drop table sharkTest5tachyon") + assert(!isTachyonTable(DEFAULT_DB_NAME, "sharkTest5tachyon")) + } + + ////////////////////////////////////////////////////////////////////////////// + // Caching Hive-partititioned tables + // Note: references to 'partition' for this section refer to a Hive-partition. + ////////////////////////////////////////////////////////////////////////////// + test("Use regular CREATE TABLE and '_tachyon' suffix to create partitioned Tachyon table") { + sc.runSql("drop table if exists empty_part_table_tachyon") + sc.runSql("""create table empty_part_table_tachyon(key int, value string) + partitioned by (keypart int)""") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_part_table_tachyon")) + + sc.runSql("drop table if exists empty_part_table_tachyon") + } + + test("Use regular CREATE TABLE and table properties to create partitioned Tachyon table") { + sc.runSql("drop table if exists empty_part_table_tachyon_tbl_props") + sc.runSql("""create table empty_part_table_tachyon_tbl_props(key int, value string) + partitioned by (keypart int) tblproperties('shark.cache' = 'tachyon')""") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_part_table_tachyon_tbl_props")) + + sc.runSql("drop table if exists empty_part_table_tachyon_tbl_props") + } + + test("alter Tachyon table by adding a new partition") { + sc.runSql("drop table if exists alter_part_tachyon") + sc.runSql("""create table alter_part_tachyon(key int, value string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_part_tachyon add partition(keypart = 1)""") + val tableName = "alter_part_tachyon" + val partitionColumn = "keypart=1" + assert(isTachyonTable(DEFAULT_DB_NAME, "alter_part_tachyon", Some(partitionColumn))) + + sc.runSql("drop table if exists alter_part_tachyon") + } + + test("alter Tachyon table by dropping a partition") { + sc.runSql("drop table if exists alter_drop_tachyon") + sc.runSql("""create table alter_drop_tachyon(key int, value string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_drop_tachyon add partition(keypart = 1)""") + + val tableName = "alter_drop_tachyon" + val partitionColumn = "keypart=1" + assert(isTachyonTable(DEFAULT_DB_NAME, "alter_drop_tachyon", Some(partitionColumn))) + sc.runSql("""alter table alter_drop_tachyon drop partition(keypart = 1)""") + assert(!isTachyonTable(DEFAULT_DB_NAME, "alter_drop_tachyon", Some(partitionColumn))) + + sc.runSql("drop table if exists alter_drop_tachyon") + } + + test("insert into a partition of a Tachyon table") { + val tableName = "insert_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 1) + expectSql("select value from insert_part_tachyon where key = 407 and keypart = 1", "val_407") + + sc.runSql("drop table if exists insert_part_tachyon") + } + + test("insert overwrite a partition of a Tachyon table") { + val tableName = "insert_over_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 1) + expectSql("""select value from insert_over_part_tachyon + where key = 407 and keypart = 1""", "val_407") + sc.runSql("""insert overwrite table insert_over_part_tachyon partition(keypart = 1) + select key, -1 from test""") + expectSql("select value from insert_over_part_tachyon where key = 407 and keypart = 1", "-1") + + sc.runSql("drop table if exists insert_over_part_tachyon") + } + + test("scan partitioned Tachyon table that's empty") { + sc.runSql("drop table if exists empty_part_table_tachyon") + sc.runSql("""create table empty_part_table_tachyon(key int, value string) + partitioned by (keypart int)""") + expectSql("select count(*) from empty_part_table_tachyon", "0") + + sc.runSql("drop table if exists empty_part_table_tachyon") + } + + test("scan partitioned Tachyon table that has a single partition") { + val tableName = "scan_single_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 1) + expectSql("select * from scan_single_part_tachyon where key = 407", "407\tval_407\t1") + + sc.runSql("drop table if exists scan_single_part_tachyon") + } + + test("scan partitioned Tachyon table that has multiple partitions") { + val tableName = "scan_mult_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 3) + expectSql("select * from scan_mult_part_tachyon where key = 407 order by keypart", + Array("407\tval_407\t1", "407\tval_407\t2", "407\tval_407\t3")) + + sc.runSql("drop table if exists scan_mult_part_tachyon") + } + + test("drop/unpersist partitioned Tachyon table that has multiple partitions") { + val tableName = "drop_mult_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 3) + expectSql("select count(1) from drop_mult_part_tachyon", "1500") + sc.runSql("drop table drop_mult_part_tachyon ") + assert(!isTachyonTable(DEFAULT_DB_NAME, tableName)) + + sc.runSql("drop table if exists drop_mult_part_tachyon") + } + + ///////////////////////////////////////////////////////////////////////////// + // LOAD for Tachyon tables + ////////////////////////////////////////////////////////////////////////////// + test ("LOAD INTO a Tachyon table") { + sc.runSql("drop table if exists load_into_tachyon") + sc.runSql("create table load_into_tachyon (key int, value string)") + sc.runSql("load data local inpath '%s' into table load_into_tachyon".format(KV1_TXT_PATH)) + expectSql("select count(*) from load_into_tachyon", "500") + + sc.runSql("drop table if exists load_into_tachyon") + } + + test ("LOAD OVERWRITE a Tachyon table") { + sc.runSql("drop table if exists load_overwrite_tachyon") + sc.runSql("create table load_overwrite_tachyon (key int, value string)") + sc.runSql("load data local inpath '%s' into table load_overwrite_tachyon". + format("${hiveconf:shark.test.data.path}/kv3.txt")) + expectSql("select count(*) from load_overwrite_tachyon", "25") + sc.runSql("load data local inpath '%s' overwrite into table load_overwrite_tachyon". + format(KV1_TXT_PATH)) + expectSql("select count(*) from load_overwrite_tachyon", "500") + sc.runSql("drop table if exists load_overwrite_tachyon") + } + + test ("LOAD INTO a partitioned Tachyon table") { + sc.runSql("drop table if exists load_into_part_tachyon") + sc.runSql("""create table load_into_part_tachyon (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""load data local inpath '%s' into table load_into_part_tachyon + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectSql("select count(*) from load_into_part_tachyon", "500") + sc.runSql("drop table if exists load_into_part_tachyon") + } + + test ("LOAD OVERWRITE a partitioned Tachyon table") { + sc.runSql("drop table if exists load_overwrite_part_tachyon") + sc.runSql("""create table load_overwrite_part_tachyon (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""load data local inpath '%s' overwrite into table load_overwrite_part_tachyon + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectSql("select count(*) from load_overwrite_part_tachyon", "500") + sc.runSql("drop table if exists load_overwrite_part_tachyon") + } + } +} From fcf5289bbee420e1885bcb04f9937bd1f4851705 Mon Sep 17 00:00:00 2001 From: Harvey Date: Tue, 14 Jan 2014 13:00:21 -0800 Subject: [PATCH 311/331] Patches for partition stats and pruning --- ...Util.scala => MapSplitPruningHelper.scala} | 0 .../scala/shark/execution/TableReader.scala | 22 ++++++------------- .../execution/optimization/ColumnPruner.scala | 3 ++- 3 files changed, 9 insertions(+), 16 deletions(-) rename src/main/scala/shark/execution/{MapSplitPruningUtil.scala => MapSplitPruningHelper.scala} (100%) diff --git a/src/main/scala/shark/execution/MapSplitPruningUtil.scala b/src/main/scala/shark/execution/MapSplitPruningHelper.scala similarity index 100% rename from src/main/scala/shark/execution/MapSplitPruningUtil.scala rename to src/main/scala/shark/execution/MapSplitPruningHelper.scala diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index d9993baa..85a6b913 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -143,8 +143,8 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } /** - * Fetch and optinally prune an RDD from the Shark metastore using each partition key given, and - * return a union of all the fetched (and possibly pruned) RDDs. + * Fetches an RDD from the Shark metastore for each partition key given. Returns a single, unioned + * RDD representing all of the specified partition keys. * * @param partitions A collection of Hive-partition metadata, such as partition columns and * partition key specifications. @@ -178,7 +178,9 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade val hivePartitionedTable = hivePartitionedTableOpt.get val rddAndStatsOpt = hivePartitionedTable.getPartitionAndStats(partitionKeyStr) - if (rddAndStatsOpt.isEmpty) throwMissingPartitionException(partitionKeyStr) + if (rddAndStatsOpt.isEmpty) { + throwMissingPartitionException(partitionKeyStr) + } val (hivePartitionRDD, hivePartitionStats) = (rddAndStatsOpt.get._1, rddAndStatsOpt.get._2) val prunedPartitionRDD = if (pruningFnOpt.isDefined) { val pruningFn = pruningFnOpt.get @@ -186,21 +188,11 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } else { hivePartitionRDD } - hivePartitionRDD.mapPartitions { iter => + prunedPartitionRDD.mapPartitions { iter => if (iter.hasNext) { - val nextElem = iter.next() - // `pruningFn` may or may not unravel the TablePartition. - val rowIter = nextElem match { - case tablePartition: TablePartition => { - tablePartition.iterator - } - case _ => { - iter - } - } // Map each tuple to a row object val rowWithPartArr = new Array[Object](2) - rowIter.map { value => + iter.map { value => rowWithPartArr.update(0, value.asInstanceOf[Object]) rowWithPartArr.update(1, partValues) rowWithPartArr.asInstanceOf[Object] diff --git a/src/main/scala/shark/execution/optimization/ColumnPruner.scala b/src/main/scala/shark/execution/optimization/ColumnPruner.scala index 06ce0186..38efb328 100644 --- a/src/main/scala/shark/execution/optimization/ColumnPruner.scala +++ b/src/main/scala/shark/execution/optimization/ColumnPruner.scala @@ -37,7 +37,8 @@ class ColumnPruner(@transient op: TopOperator[_], @transient tbl: Table) extends val columnsUsed: BitSet = { val colsToKeep = computeColumnsToKeep() - val allColumns = tbl.getAllCols().map(x => x.getName()) + // No need to prune partition columns - Hive does that for us. + val allColumns = tbl.getCols().map(x => x.getName()) val b = new BitSet() for (i <- Range(0, allColumns.size) if colsToKeep.contains(allColumns(i))) { b.set(i, true) From 159edf41222e01af6e3b0f2b70e07ddd6817f394 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 14 Jan 2014 13:35:22 -0800 Subject: [PATCH 312/331] Upgrade Spark API. --- .../scala/shark/execution/GroupByPostShuffleOperator.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala index df00112b..bc3cfb92 100755 --- a/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala +++ b/src/main/scala/shark/execution/GroupByPostShuffleOperator.scala @@ -221,7 +221,8 @@ class GroupByPostShuffleOperator extends GroupByPreShuffleOperator // No distinct keys. val aggregator = new Aggregator[Any, Any, ArrayBuffer[Any]]( GroupByAggregator.createCombiner _, GroupByAggregator.mergeValue _, null) - val hashedRdd = repartitionedRDD.mapPartitions(aggregator.combineValuesByKey(_), + val hashedRdd = repartitionedRDD.mapPartitionsWithContext( + (context, iter) => aggregator.combineValuesByKey(iter, context), preservesPartitioning = true) val op = OperatorSerializationWrapper(this) From 5befd914e785360cd3042e708e9d871261926181 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Wed, 15 Jan 2014 10:41:17 +0800 Subject: [PATCH 313/331] add script support for shark with spark on yarn. --- conf/shark-env.sh.template | 5 +++++ run | 20 ++++++++++++++++++++ 2 files changed, 25 insertions(+) diff --git a/conf/shark-env.sh.template b/conf/shark-env.sh.template index a7a42cc9..deb14f2b 100755 --- a/conf/shark-env.sh.template +++ b/conf/shark-env.sh.template @@ -39,6 +39,11 @@ export HIVE_HOME="" # Only required if using Mesos: #export MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos.so +# Only required if run shark with spark on yarn +#export SHARK_EXEC_MODE=yarn +#export SPARK_ASSEMBLY_JAR= +#export SHARK_ASSEMBLY_JAR= + # (Optional) Extra classpath #export SPARK_LIBRARY_PATH="" diff --git a/run b/run index f24c201d..83294608 100755 --- a/run +++ b/run @@ -47,6 +47,26 @@ if [ -n "$MASTER" ] ; then fi fi +# check for shark with spark on yarn params +if [ "x$SHARK_EXEC_MODE" == "xyarn" ] ; then + if [ "x$SPARK_ASSEMBLY_JAR" == "x" ] ; then + echo "No SPARK_ASSEMBLY_JAR specified. Please set SPARK_ASSEMBLY_JAR for spark on yarn mode." + exit 1 + else + export SPARK_JAR=$SPARK_ASSEMBLY_JAR + fi + + if [ "x$SHARK_ASSEMBLY_JAR" == "x" ] ; then + echo "No SHARK_ASSEMBLY_JAR specified. please set SHARK_ASSEMBLY_JAR for spark on yarn mode." + exit 1 + else + export SPARK_YARN_APP_JAR = $SHARK_ASSEMBLY_JAR + fi + + # use yarn-client mode for interactive shell. + export MASTER=yarn-client +fi + # Check for optionally specified configuration file path if [ "x$HIVE_CONF_DIR" == "x" ] ; then HIVE_CONF_DIR="$HIVE_HOME/conf" From b1cefe799cbd758f7481eedf5765be5f82dc4a1a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 14 Jan 2014 23:11:01 -0800 Subject: [PATCH 314/331] TGF Scala 2.10 upgrade. --- .../resources/tablerdd/rddtable_generator.py | 19 +- src/main/scala/shark/api/ClassTags.scala | 14 ++ src/main/scala/shark/api/DataTypes.java | 15 +- src/main/scala/shark/api/RDDTable.scala | 201 +++++++++-------- src/main/scala/shark/tgf/TGF.scala | 209 ++++++++++-------- 5 files changed, 249 insertions(+), 209 deletions(-) create mode 100644 src/main/scala/shark/api/ClassTags.scala diff --git a/src/main/resources/tablerdd/rddtable_generator.py b/src/main/resources/tablerdd/rddtable_generator.py index 8b359d82..f5c2e291 100755 --- a/src/main/resources/tablerdd/rddtable_generator.py +++ b/src/main/resources/tablerdd/rddtable_generator.py @@ -46,10 +46,11 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind // *** This file is auto-generated from RDDTable_generator.py *** +import scala.reflect.ClassTag import org.apache.spark.rdd.RDD object RDDTableImplicits { - private type M[T] = ClassManifest[T] + private type C[T] = ClassTag[T] """ @@ -62,7 +63,7 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind implicit def rddToTable$num[$tmlist] (rdd: RDD[($tlist)]): RDDTableFunctions = RDDTable(rdd) -""").substitute(num = x, tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) +""").substitute(num = x, tmlist = createList(1, x, "T", ": C", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) p.write(tableClass) prefix = """ @@ -70,25 +71,25 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind object RDDTable { - private type M[T] = ClassManifest[T] - private def m[T](implicit m : ClassManifest[T]) = classManifest[T](m) + private type C[T] = ClassTag[T] + private def ct[T](implicit c: ClassTag[T]) = c """ p.write(prefix) - + for x in range(2,23): tableClass = Template( """ def apply[$tmlist] (rdd: RDD[($tlist)]) = { - val cm = implicitly[Manifest[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(cm) + val classTag = implicitly[ClassTag[Seq[Any]]] + val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) new RDDTableFunctions(rddSeq, Seq($mtlist)) } -""").substitute(tmlist = createList(1, x, "T", ": M", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4), - mtlist = createList(1, x, "m[T", "]", ", ", indent=4)) +""").substitute(tmlist = createList(1, x, "T", ": C", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4), + mtlist = createList(1, x, "ct[T", "]", ", ", indent=4)) p.write(tableClass) diff --git a/src/main/scala/shark/api/ClassTags.scala b/src/main/scala/shark/api/ClassTags.scala new file mode 100644 index 00000000..b3c17d01 --- /dev/null +++ b/src/main/scala/shark/api/ClassTags.scala @@ -0,0 +1,14 @@ +package shark.api + +import scala.reflect.classTag + +object ClassTags { + // List of primitive ClassTags. + val jBoolean = classTag[java.lang.Boolean] + val jByte = classTag[java.lang.Byte] + val jShort = classTag[java.lang.Short] + val jInt = classTag[java.lang.Integer] + val jLong = classTag[java.lang.Long] + val jFloat = classTag[java.lang.Float] + val jDouble = classTag[java.lang.Double] +} diff --git a/src/main/scala/shark/api/DataTypes.java b/src/main/scala/shark/api/DataTypes.java index 8190e55c..4c5ec3f9 100644 --- a/src/main/scala/shark/api/DataTypes.java +++ b/src/main/scala/shark/api/DataTypes.java @@ -26,6 +26,7 @@ import scala.reflect.ClassTag$; import org.apache.hadoop.hive.serde.Constants; +import shark.Utils$; /** * List of data types defined in Shark APIs. @@ -108,19 +109,19 @@ public static DataType fromHiveType(String hiveType) throws UnknownDataTypeExcep } public static DataType fromClassTag(ClassTag m) throws UnknownDataTypeException { - if (m.equals(ClassTag$.MODULE$.Boolean())) { + if (m.equals(ClassTag$.MODULE$.Boolean()) || m.equals(ClassTags$.MODULE$.jBoolean())) { return INT; - } else if (m.equals(ClassTag$.MODULE$.Byte())){ + } else if (m.equals(ClassTag$.MODULE$.Byte()) || m.equals(ClassTags$.MODULE$.jByte())){ return TINYINT; - } else if (m.equals(ClassTag$.MODULE$.Short())) { + } else if (m.equals(ClassTag$.MODULE$.Short()) || m.equals(ClassTags$.MODULE$.jShort())) { return SMALLINT; - } else if (m.equals(ClassTag$.MODULE$.Int())) { + } else if (m.equals(ClassTag$.MODULE$.Int()) || m.equals(ClassTags$.MODULE$.jInt())) { return INT; - } else if (m.equals(ClassTag$.MODULE$.Long())) { + } else if (m.equals(ClassTag$.MODULE$.Long()) || m.equals(ClassTags$.MODULE$.jLong())) { return BIGINT; - } else if (m.equals(ClassTag$.MODULE$.Float())) { + } else if (m.equals(ClassTag$.MODULE$.Float()) || m.equals(ClassTags$.MODULE$.jFloat())) { return FLOAT; - } else if (m.equals(ClassTag$.MODULE$.Double())) { + } else if (m.equals(ClassTag$.MODULE$.Double()) || m.equals(ClassTags$.MODULE$.jDouble())) { return DOUBLE; } else if (m.equals(ClassTag$.MODULE$.apply(String.class))) { return STRING; diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index 1485b2eb..fa3429a1 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -1,4 +1,3 @@ - /* * Copyright (C) 2012 The Regents of The University California. * All rights reserved. @@ -20,10 +19,8 @@ package shark.api // *** This file is auto-generated from RDDTable_generator.py *** -import scala.language.implicitConversions - -import org.apache.spark.rdd.RDD import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD object RDDTableImplicits { private type C[T] = ClassTag[T] @@ -65,72 +62,72 @@ object RDDTableImplicits { (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable11[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C] + implicit def rddToTable11[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable12[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C] + implicit def rddToTable12[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable13[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C] + implicit def rddToTable13[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable14[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C] + implicit def rddToTable14[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable15[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C] + implicit def rddToTable15[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable16[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C] + implicit def rddToTable16[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable17[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable17[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable18[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable18[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable19[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable19[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable20[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, - T20: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable20[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable21[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, - T20: C, T21: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable21[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21)]): RDDTableFunctions = RDDTable(rdd) - implicit def rddToTable22[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, - T20: C, T21: C, T22: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + implicit def rddToTable22[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C, T22: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22)]): RDDTableFunctions = RDDTable(rdd) @@ -139,7 +136,7 @@ object RDDTableImplicits { object RDDTable { private type C[T] = ClassTag[T] - private def ct[T](implicit c : ClassTag[T]) = c + private def ct[T](implicit c: ClassTag[T]) = c def apply[T1: C, T2: C] (rdd: RDD[(T1, T2)]) = { @@ -209,142 +206,142 @@ object RDDTable { (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C] + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C] (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], - ct[T18])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], - ct[T18], ct[T19])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, - T20: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], - ct[T18], ct[T19], ct[T20])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19], ct[T20])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, - T20: C, T21: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], - ct[T18], ct[T19], ct[T20], ct[T21])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19], ct[T20], ct[T21])) } - def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, - T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, - T20: C, T21: C, T22: C] - (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + def apply[T1: C, T2: C, T3: C, T4: C, T5: C, T6: C, T7: C, T8: C, T9: C, T10: C, + T11: C, T12: C, T13: C, T14: C, T15: C, T16: C, T17: C, T18: C, T19: C, + T20: C, T21: C, T22: C] + (rdd: RDD[(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22)]) = { val classTag = implicitly[ClassTag[Seq[Any]]] val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], - ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], - ct[T18], ct[T19], ct[T20], ct[T21], ct[T22])) + new RDDTableFunctions(rddSeq, Seq(ct[T1], ct[T2], ct[T3], ct[T4], ct[T5], ct[T6], ct[T7], ct[T8], ct[T9], + ct[T10], ct[T11], ct[T12], ct[T13], ct[T14], ct[T15], ct[T16], ct[T17], + ct[T18], ct[T19], ct[T20], ct[T21], ct[T22])) } } diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index 311fc872..fe3a10eb 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -20,48 +20,54 @@ package shark.tgf import java.sql.Timestamp import java.util.Date +import scala.reflect.{classTag, ClassTag} import scala.util.parsing.combinator._ import org.apache.spark.rdd.RDD import shark.api._ import shark.SharkContext +import java.lang.reflect.Method /** * This object is responsible for handling TGF (Table Generating Function) commands. * + * {{{ * -- TGF Commands -- * GENERATE tgfname(param1, param2, ... , param_n) * GENERATE tgfname(param1, param2, ... , param_n) AS tablename + * }}} * - * Parameters can either be of primitive types, e.g. int, or of type RDD[Product]. - * TGF.execute() will use reflection looking for an object of name "tgfname", invoking apply() with the primitive - * values. If the type of a parameter to apply() is RDD[Product], it will assume the parameter is the name of a table, - * which it will turn into an RDD before invoking apply(). + * Parameters can either be of primitive types, eg int, or of type RDD[Product]. TGF.execute() + * will use reflection looking for an object of name "tgfname", invoking apply() with the + * primitive values. If the type of a parameter to apply() is RDD[Product], it will assume the + * parameter is the name of a table, which it will turn into an RDD before invoking apply(). * - * For example, "GENERATE MyObj(25, emp)" will invoke MyObj.apply(25, sc.sql2rdd("select * from emp")), assuming - * the TGF object (MyObj) has an apply function that takes an int and an RDD[Product]. + * For example, "GENERATE MyObj(25, emp)" will invoke + * MyObj.apply(25, sc.sql2rdd("select * from emp")) + * , assuming the TGF object (MyObj) has an apply function that takes an int and an RDD[Product]. * - * The "as" version of the command saves the output in a new table named "tablename", whereas the other version - * returns a ResultSet + * The "as" version of the command saves the output in a new table named "tablename", + * whereas the other version returns a ResultSet. * * -- Defining TGF objects -- - * TGF objects need to have an apply() function and take an arbitrary number of either primitive or RDD[Product] typed - * parameters. The apply() function should either return an RDD[Product] or RDDSchema. When the former case is used, - * the returned table's schema and column names need to be defined through a Java annotation called @Schema. Here is - * a short example: - * + * TGF objects need to have an apply() function and take an arbitrary number of either primitive + * or RDD[Product] typed parameters. The apply() function should either return an RDD[Product] + * or RDDSchema. When the former case is used, the returned table's schema and column names need + * to be defined through a Java annotation called @Schema. Here is a short example: + * {{{ * object MyTGF1 { * \@Schema(spec = "name string, age int") * def apply(table1: RDD[(String, String, Int)]): RDD[Product] = { * // code that manipulates table1 and returns a new RDD of tuples * } * } + * }}} * - * Sometimes, the TGF dynamically determines the number or types of columns returned. In this case, the TGF can - * use the RDDSchema return type instead of Java annotations. RDDSchema simply contains a schema string and an RDD - * of results. For example: - * + * Sometimes, the TGF dynamically determines the number or types of columns returned. In this case, + * the TGF can use the RDDSchema return type instead of Java annotations. RDDSchema simply contains + * a schema string and an RDD of results. For example: + * {{{ * object MyTGF2 { * \@Schema(spec = "name string, age int") * def apply(table1: RDD[(String, String, Int)]): RDD[Product] = { @@ -69,30 +75,39 @@ import shark.SharkContext * return RDDSchema(rdd.asInstanceOf[RDD[Seq[_]]], "name string, age int") * } * } + * }}} * - * Sometimes the TGF needs to internally make SQL calls. For that, it needs access to a SharkContext object. Therefore, - * def apply(sc: SharkContext, table1: RDD[(String, String, Int)]): RDD[Product] = { - * // code that can use sc, for example by calling sc.sql2rdd() - * // code that manipulates table1 and returns a new RDD of tuples - * } + * Sometimes the TGF needs to internally make SQL calls. For that, it needs access to a + * SharkContext object. Therefore, + * {{{ + * def apply(sc: SharkContext, table1: RDD[(String, String, Int)]): RDD[Product] = { + * // code that can use sc, for example by calling sc.sql2rdd() + * // code that manipulates table1 and returns a new RDD of tuples + * } + * }}} */ object TGF { private val parser = new TGFParser /** - * Executes a TGF command and gives back the ResultSet. Mainly to be used from SharkContext (e.g. runSql()) + * Executes a TGF command and gives back the ResultSet. + * Mainly to be used from SharkContext (e.g. runSql()) + * * @param sql TGF command, e.g. "GENERATE name(params) AS tablename" * @param sc SharkContext * @return ResultSet containing the results of the command */ def execute(sql: String, sc: SharkContext): ResultSet = { - val ast = parser.parseAll(parser.tgf, sql).getOrElse{throw new QueryExecutionException("TGF parse error: "+ sql)} + val ast = parser.parseAll(parser.tgf, sql).getOrElse( + throw new QueryExecutionException("TGF parse error: "+ sql)) val (tableNameOpt, tgfName, params) = ast match { - case Tuple2(tgfName, params) => (None, tgfName.asInstanceOf[String], params.asInstanceOf[List[String]]) - case Tuple3(tableName, tgfName, params) => (Some(tableName.asInstanceOf[String]), tgfName.asInstanceOf[String], - params.asInstanceOf[List[String]]) + case (tgfName, params) => + (None, tgfName.asInstanceOf[String], params.asInstanceOf[List[String]]) + case (tableName, tgfName, params) => + (Some(tableName.asInstanceOf[String]), tgfName.asInstanceOf[String], + params.asInstanceOf[List[String]]) } val obj = reflectInvoke(tgfName, params, sc) @@ -100,54 +115,58 @@ object TGF { val (sharkSchema, resultArr) = tableNameOpt match { case Some(tableName) => // materialize results - val helper = new RDDTableFunctions(rdd, schema.map{ case (_, tpe) => toManifest(tpe)}) + val helper = new RDDTableFunctions(rdd, schema.map { case (_, tpe) => toClassTag(tpe) }) helper.saveAsTable(tableName, schema.map{ case (name, _) => name}) (Array[ColumnDesc](), Array[Array[Object]]()) - case None => // return results - val newSchema = schema.map{ case (name, tpe) => new ColumnDesc(name, DataTypes.fromManifest(toManifest(tpe)))} + val newSchema = schema.map { case (name, tpe) => + new ColumnDesc(name, DataTypes.fromClassTag(toClassTag(tpe))) + } val res = rdd.collect().map{p => p.map( _.asInstanceOf[Object] ).toArray} (newSchema.toArray, res) } new ResultSet(sharkSchema, resultArr) } - private def getMethod(tgfName: String, methodName: String) = { + private def getMethod(tgfName: String, methodName: String): Option[Method] = { val tgfClazz = try { Thread.currentThread().getContextClassLoader.loadClass(tgfName) } catch { - case ex: ClassNotFoundException => throw new QueryExecutionException("Couldn't find TGF class: " + tgfName) + case ex: ClassNotFoundException => + throw new QueryExecutionException("Couldn't find TGF class: " + tgfName) } val methods = tgfClazz.getDeclaredMethods.filter(_.getName == methodName) if (methods.isEmpty) None else Some(methods(0)) } - private def getSchema(tgfOutput: Object, tgfName: String): Tuple2[RDD[Seq[_]], Seq[Tuple2[String,String]]] = { - if (tgfOutput.isInstanceOf[RDDSchema]) { - val rddSchema = tgfOutput.asInstanceOf[RDDSchema] - val schema = parser.parseAll(parser.schema, rddSchema.schema) - - (rddSchema.rdd, schema.get) - } else if (tgfOutput.isInstanceOf[RDD[Product]]) { - val applyMethod = getMethod(tgfName, "apply") - if (applyMethod == None) { - throw new QueryExecutionException("TGF lacking apply() method") - } - - val annotations = applyMethod.get.getAnnotation(classOf[Schema]) - if (annotations == null || annotations.spec() == null) { - throw new QueryExecutionException("No schema annotation found for TGF") - } - - val schema = parser.parseAll(parser.schema, annotations.spec()) - if (schema == None) { - throw new QueryExecutionException("Error parsing TGF schema annotation (@Schema(spec=...)") - } - - (tgfOutput.asInstanceOf[RDD[Product]].map(_.productIterator.toList), schema.get) - } else { - throw new QueryExecutionException("TGF output needs to be of type RDD or RDDSchema") + private def getSchema(tgfOutput: Object, tgfName: String): (RDD[Seq[_]], Seq[(String,String)]) = { + tgfOutput match { + case rddSchema: RDDSchema => + val schema = parser.parseAll(parser.schema, rddSchema.schema) + + (rddSchema.rdd, schema.get) + case rdd: RDD[Product] => + val applyMethod = getMethod(tgfName, "apply") + if (applyMethod == None) { + throw new QueryExecutionException("TGF lacking apply() method") + } + + val annotations = applyMethod.get.getAnnotation(classOf[Schema]) + if (annotations == null || annotations.spec() == null) { + throw new QueryExecutionException("No schema annotation found for TGF") + } + + // TODO: How can we compare schema with None? + val schema = parser.parseAll(parser.schema, annotations.spec()) + if (schema.isEmpty) { + throw new QueryExecutionException( + "Error parsing TGF schema annotation (@Schema(spec=...)") + } + + (rdd.map(_.productIterator.toList), schema.get) + case _ => + throw new QueryExecutionException("TGF output needs to be of type RDD or RDDSchema") } } @@ -162,11 +181,12 @@ object TGF { val typeNames: Seq[String] = applyMethod.getParameterTypes.toList.map(_.toString) - val augParams = if (!typeNames.isEmpty && typeNames.head.startsWith("class shark.SharkContext")) { - Seq("sc") ++ paramStrs - } else { - paramStrs - } + val augParams = + if (!typeNames.isEmpty && typeNames.head.startsWith("class shark.SharkContext")) { + Seq("sc") ++ paramStrs + } else { + paramStrs + } if (augParams.length != typeNames.length) { throw new QueryExecutionException("Expecting " + typeNames.length + @@ -174,32 +194,39 @@ object TGF { } val params = (augParams.toList zip typeNames.toList).map { - case (param: String, tpe: String) if (tpe.startsWith("class shark.SharkContext")) => sc - case (param: String, tpe: String) if (tpe.startsWith("class org.apache.spark.rdd.RDD")) => tableRdd(sc, param) - case (param: String, tpe: String) if (tpe.startsWith("long")) => param.toLong - case (param: String, tpe: String) if (tpe.startsWith("int")) => param.toInt - case (param: String, tpe: String) if (tpe.startsWith("double")) => param.toDouble - case (param: String, tpe: String) if (tpe.startsWith("float")) => param.toFloat - case (param: String, tpe: String) if (tpe.startsWith("class java.lang.String") || - tpe.startsWith("class String")) => param.stripPrefix("\"").stripSuffix("\"") - case (param: String, tpe: String) => throw - new QueryExecutionException("Expected TGF parameter type: " + tpe + " (" + param + ")") + case (param: String, tpe: String) if tpe.startsWith("class shark.SharkContext") => + sc + case (param: String, tpe: String) if tpe.startsWith("class org.apache.spark.rdd.RDD") => + tableRdd(sc, param) + case (param: String, tpe: String) if tpe.startsWith("long") => + param.toLong + case (param: String, tpe: String) if tpe.startsWith("int") => + param.toInt + case (param: String, tpe: String) if tpe.startsWith("double") => + param.toDouble + case (param: String, tpe: String) if tpe.startsWith("float") => + param.toFloat + case (param: String, tpe: String) if tpe.startsWith("class java.lang.String") || + tpe.startsWith("class String") => + param.stripPrefix("\"").stripSuffix("\"") + case (param: String, tpe: String) => + throw new QueryExecutionException(s"Expected TGF parameter type: $tpe ($param)") } - applyMethod.invoke(null, params.asInstanceOf[List[Object]]:_*) + applyMethod.invoke(null, params.asInstanceOf[List[Object]] : _*) } - private def toManifest(tpe: String): ClassManifest[_] = { - if (tpe == "boolean") classManifest[java.lang.Boolean] - else if (tpe == "tinyint") classManifest[java.lang.Byte] - else if (tpe == "smallint") classManifest[java.lang.Short] - else if (tpe == "int") classManifest[java.lang.Integer] - else if (tpe == "bigint") classManifest[java.lang.Long] - else if (tpe == "float") classManifest[java.lang.Float] - else if (tpe == "double") classManifest[java.lang.Double] - else if (tpe == "string") classManifest[java.lang.String] - else if (tpe == "timestamp") classManifest[Timestamp] - else if (tpe == "date") classManifest[Date] + private def toClassTag(tpe: String): ClassTag[_] = { + if (tpe == "boolean") classTag[Boolean] + else if (tpe == "tinyint") classTag[Byte] + else if (tpe == "smallint") classTag[Short] + else if (tpe == "int") classTag[Integer] + else if (tpe == "bigint") classTag[Long] + else if (tpe == "float") classTag[Float] + else if (tpe == "double") classTag[Double] + else if (tpe == "string") classTag[String] + else if (tpe == "timestamp") classTag[Timestamp] + else if (tpe == "date") classTag[Date] else { throw new QueryExecutionException("Unknown column type specified in schema (" + tpe + ")") } @@ -238,7 +265,8 @@ case class RDDSchema(rdd: RDD[Seq[_]], schema: String) private class TGFParser extends JavaTokenParsers { - /* Code to enable case-insensitive modifiers to strings, e.g. "DataBricks".ci will match "databricks" */ + // Code to enable case-insensitive modifiers to strings, e.g. + // "DataBricks".ci will match "databricks" class MyString(str: String) { def ci: Parser[String] = ("(?i)" + str).r } @@ -250,7 +278,7 @@ private class TGFParser extends JavaTokenParsers { /** * @return Tuple2 containing a TGF method name and a List of parameters as strings */ - def basicTgf: Parser[Tuple2[String, List[String]]] = { + def basicTgf: Parser[(String, List[String])] = { ("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")") ^^ { case id1 ~ x => (id1, x.asInstanceOf[List[String]]) } } @@ -258,18 +286,17 @@ private class TGFParser extends JavaTokenParsers { /** * @return Tuple3 containing a table name, TGF method name and a List of parameters as strings */ - def saveTgf: Parser[Tuple3[String, String, List[String]]] = { + def saveTgf: Parser[(String, String, List[String])] = { (("GENERATE".ci ~> methodName) ~ (("(" ~> repsep(param, ",")) <~ ")")) ~ (("AS".ci) ~> ident) ^^ { case id1 ~ x ~ id2 => (id2, id1, x.asInstanceOf[List[String]]) } } - def schema: Parser[Seq[Tuple2[String,String]]] = repsep(nameType, ",") + def schema: Parser[Seq[(String,String)]] = repsep(nameType, ",") - def nameType: Parser[Tuple2[String,String]] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } + def nameType: Parser[(String,String)] = ident ~ ident ^^ { case name~tpe => Tuple2(name, tpe) } def param: Parser[Any] = stringLiteral | floatingPointNumber | decimalNumber | ident | failure("Expected a string, number, or identifier as parameters in TGF") - def methodName: Parser[String] = - """[a-zA-Z_][\w\.]*""".r + def methodName: Parser[String] = """[a-zA-Z_][\w\.]*""".r } From b29dae2e7e6b99febcb326291c0c2e188c92b82b Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 15 Jan 2014 00:54:42 -0800 Subject: [PATCH 315/331] Some docs for MemoryTable, updates based on review. --- .../shark/execution/MapSplitPruning.scala | 20 +++++------ .../scala/shark/memstore2/MemoryTable.scala | 33 +++++++++++++------ src/test/scala/shark/SharkRunner.scala | 5 +-- 3 files changed, 36 insertions(+), 22 deletions(-) diff --git a/src/main/scala/shark/execution/MapSplitPruning.scala b/src/main/scala/shark/execution/MapSplitPruning.scala index 4de38fc0..976ebf50 100644 --- a/src/main/scala/shark/execution/MapSplitPruning.scala +++ b/src/main/scala/shark/execution/MapSplitPruning.scala @@ -181,16 +181,16 @@ object MapSplitPruning { } } - private def getIDStructField(field: StructField): IDStructField = { - field match { - case myField: MyField => { - MapSplitPruningHelper.getStructFieldFromUnionOIField(myField) - .asInstanceOf[IDStructField] - } - case idStructField: IDStructField => idStructField - case otherFieldType: Any => { - throw new Exception("Unrecognized StructField: " + otherFieldType) - } + private def getIDStructField(field: StructField): IDStructField = field match { + case myField: MyField => { + // For partitioned tables, the ColumnarStruct's IDStructFields are enclosed inside + // the Hive UnionStructObjectInspector's MyField objects. + MapSplitPruningHelper.getStructFieldFromUnionOIField(myField) + .asInstanceOf[IDStructField] + } + case idStructField: IDStructField => idStructField + case otherFieldType: Any => { + throw new Exception("Unrecognized StructField: " + otherFieldType) } } } diff --git a/src/main/scala/shark/memstore2/MemoryTable.scala b/src/main/scala/shark/memstore2/MemoryTable.scala index bfd5163d..1a971d4c 100644 --- a/src/main/scala/shark/memstore2/MemoryTable.scala +++ b/src/main/scala/shark/memstore2/MemoryTable.scala @@ -33,32 +33,45 @@ private[shark] class MemoryTable( cacheMode: CacheType.CacheType) extends Table(databaseName, tableName, cacheMode) { - var rddValueOpt: Option[RDDValue] = None + private var _rddValueOpt: Option[RDDValue] = None + /** + * Sets the RDD and stats fields the `_rddValueOpt`. Used for INSERT/LOAD OVERWRITE. + * @param newRDD The table's data. + * @param newStats Stats for each TablePartition in `newRDD`. + * @return The previous (RDD, stats) pair for this table. + */ def put( newRDD: RDD[TablePartition], newStats: collection.Map[Int, TablePartitionStats] = new HashMap[Int, TablePartitionStats]() ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { - val prevRDDAndStatsOpt = rddValueOpt.map(_.toTuple) - if (rddValueOpt.isDefined) { - rddValueOpt.foreach { rddValue => + val prevRDDAndStatsOpt = _rddValueOpt.map(_.toTuple) + if (_rddValueOpt.isDefined) { + _rddValueOpt.foreach { rddValue => rddValue.rdd = newRDD rddValue.stats = newStats } } else { - rddValueOpt = Some(new RDDValue(newRDD, newStats)) + _rddValueOpt = Some(new RDDValue(newRDD, newStats)) } prevRDDAndStatsOpt } + /** + * Used for append operations, such as INSERT and LOAD INTO. + * + * @param newRDD Data to append to the table. + * @param newStats Stats for each TablePartition in `newRDD`. + * @return The previous (RDD, stats) pair for this table. + */ def update( newRDD: RDD[TablePartition], newStats: Buffer[(Int, TablePartitionStats)] ): Option[(RDD[TablePartition], collection.Map[Int, TablePartitionStats])] = { - val prevRDDAndStatsOpt = rddValueOpt.map(_.toTuple) - if (rddValueOpt.isDefined) { + val prevRDDAndStatsOpt = _rddValueOpt.map(_.toTuple) + if (_rddValueOpt.isDefined) { val (prevRDD, prevStats) = (prevRDDAndStatsOpt.get._1, prevRDDAndStatsOpt.get._2) - val updatedRDDValue = rddValueOpt.get + val updatedRDDValue = _rddValueOpt.get updatedRDDValue.rdd = RDDUtils.unionAndFlatten(prevRDD, newRDD) updatedRDDValue.stats = Table.mergeStats(newStats, prevStats).toMap } else { @@ -67,8 +80,8 @@ private[shark] class MemoryTable( prevRDDAndStatsOpt } - def getRDD = rddValueOpt.map(_.rdd) + def getRDD = _rddValueOpt.map(_.rdd) - def getStats = rddValueOpt.map(_.stats) + def getStats = _rddValueOpt.map(_.stats) } diff --git a/src/test/scala/shark/SharkRunner.scala b/src/test/scala/shark/SharkRunner.scala index 5ea8047d..754737b9 100644 --- a/src/test/scala/shark/SharkRunner.scala +++ b/src/test/scala/shark/SharkRunner.scala @@ -32,7 +32,7 @@ object SharkRunner { var sharkMetastore: MemoryMetadataManager = _ - def init(): SharkContext = { + def init(): SharkContext = synchronized { if (sc == null) { sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) @@ -55,7 +55,7 @@ object SharkRunner { * Tables accessible by any test. Their properties should remain constant across * tests. */ - def loadTables() { + def loadTables() = synchronized { require(sc != null, "call init() to instantiate a SharkContext first") // Use the default namespace @@ -101,6 +101,7 @@ object SharkRunner { sc.sql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/test1.txt' INTO TABLE test1") sc.sql("drop table if exists test1_cached") sc.sql("CREATE TABLE test1_cached AS SELECT * FROM test1") + Unit } def expectSql(sql: String, expectedResults: Array[String], sort: Boolean = true) { From 1a8267a426c940dd53dd0bdf4d49c59a6a1e587c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 15 Jan 2014 14:18:35 -0800 Subject: [PATCH 316/331] Remove extraneous line in TachyonUtilImpl --- src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala b/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala index 49b64c4e..3ee76593 100644 --- a/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala +++ b/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala @@ -122,7 +122,6 @@ class TachyonUtilImpl( // The first path is just "{tableDirectory}/", so ignore it. val rawTableFiles = files.subList(1, files.size) val tableRDDsAndStats = rawTableFiles.map { filePath => - val serializedMetadata2 = client.getRawTable(filePath).getMetadata val serializedMetadata = client.getRawTable(client.getFileId(filePath)).getMetadata val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( serializedMetadata.array()) From a80a160650462c974dd7f138465a339b43682a8b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 15 Jan 2014 16:43:22 -0800 Subject: [PATCH 317/331] Two minor style update. --- src/main/scala/shark/tgf/TGF.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index fe3a10eb..cc3aceaa 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -266,7 +266,7 @@ case class RDDSchema(rdd: RDD[Seq[_]], schema: String) private class TGFParser extends JavaTokenParsers { // Code to enable case-insensitive modifiers to strings, e.g. - // "DataBricks".ci will match "databricks" + // "Berkeley".ci will match "berkeley" class MyString(str: String) { def ci: Parser[String] = ("(?i)" + str).r } From c961b5627dbc5af18680dd07f4bd331775436a4e Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 15 Jan 2014 17:12:47 -0800 Subject: [PATCH 318/331] Merge MemoryTableReader and TableReader, plus misc updates based on review. --- .../shark/execution/HadoopTableReader.scala | 10 ++- .../execution/MemoryStoreSinkOperator.scala | 13 ++-- .../shark/execution/OperatorFactory.scala | 4 +- .../scala/shark/execution/TableReader.scala | 67 +++++-------------- .../shark/parse/SharkSemanticAnalyzer.scala | 6 +- 5 files changed, 32 insertions(+), 68 deletions(-) diff --git a/src/main/scala/shark/execution/HadoopTableReader.scala b/src/main/scala/shark/execution/HadoopTableReader.scala index 1d359d64..11a0b3e7 100644 --- a/src/main/scala/shark/execution/HadoopTableReader.scala +++ b/src/main/scala/shark/execution/HadoopTableReader.scala @@ -56,7 +56,10 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf def hiveConf = _broadcastedHiveConf.value.value - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = + override def makeRDDForTable( + hiveTable: HiveTable, + pruningFnOpt: Option[PruningFunctionType] = None + ): RDD[_] = makeRDDForTable( hiveTable, _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], @@ -107,7 +110,10 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf deserializedHadoopRDD } - override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { + override def makeRDDForPartitionedTable( + partitions: Seq[HivePartition], + pruningFnOpt: Option[PruningFunctionType] = None + ): RDD[_] = { val partitionToDeserializer = partitions.map(part => (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) diff --git a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala index bf04af6c..3fece24e 100644 --- a/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala +++ b/src/main/scala/shark/execution/MemoryStoreSinkOperator.scala @@ -64,7 +64,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Whether to compose a UnionRDD from the output RDD and a previous RDD. For example, for an // INSERT INTO command, the previous RDD will contain the contents of the 'tableName'. - @transient var useUnionRDD: Boolean = _ + @transient var isInsertInto: Boolean = _ // The number of columns in the schema for the table corresponding to 'tableName'. Used only // to create a TachyonTableWriter, if Tachyon is used. @@ -123,11 +123,6 @@ class MemoryStoreSinkOperator extends TerminalOperator { val isHivePartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( databaseName, tableName) - // If true, a UnionRDD will be used to combine the RDD that contains the query output with the - // previous RDD, which is fetched using 'tableName' or - if the table is Hive-partitioned - a - // ('tableName', 'hivePartitionKey') pair. - var hasPreviousRDDForUnion = false - if (tachyonWriter != null) { // Put the table in Tachyon. op.logInfo("Putting RDD for %s.%s in Tachyon".format(databaseName, tableName)) @@ -156,7 +151,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { // Put the table in Spark block manager or Tachyon. op.logInfo("Putting %sRDD for %s.%s in %s store".format( - if (useUnionRDD) "Union" else "", + if (isInsertInto) "Union" else "", databaseName, tableName, if (cacheMode == CacheType.NONE) "disk" else cacheMode.toString)) @@ -165,7 +160,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( databaseName, tableName).get outputRDD.setName("%s.%s(%s)".format(databaseName, tableName, hivePartitionKey)) - if (useUnionRDD && hasPreviousRDDForUnion) { + if (isInsertInto) { // An RDD for the Hive partition already exists, so update its metadata entry in // 'partitionedTable'. assert(outputRDD.isInstanceOf[UnionRDD[_]]) @@ -182,7 +177,7 @@ class MemoryStoreSinkOperator extends TerminalOperator { val memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(databaseName, tableName) .getOrElse(SharkEnv.memoryMetadataManager.createMemoryTable( databaseName, tableName, cacheMode)) - if (useUnionRDD) { + if (isInsertInto) { memoryTable.update(outputRDD, statsAcc.value) } else { memoryTable.put(outputRDD, statsAcc.value.toMap) diff --git a/src/main/scala/shark/execution/OperatorFactory.scala b/src/main/scala/shark/execution/OperatorFactory.scala index a6c9323c..dd27e66d 100755 --- a/src/main/scala/shark/execution/OperatorFactory.scala +++ b/src/main/scala/shark/execution/OperatorFactory.scala @@ -49,7 +49,7 @@ object OperatorFactory extends LogHelper { numColumns: Int, hivePartitionKey: String, cacheMode: CacheType, - useUnionRDD: Boolean): TerminalOperator = { + isInsertInto: Boolean): TerminalOperator = { // TODO the terminal operator is the FileSinkOperator in Hive? val hiveOp = hiveTerminalOp.asInstanceOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] val sinkOp = _newOperatorInstance( @@ -60,7 +60,7 @@ object OperatorFactory extends LogHelper { sinkOp.numColumns = numColumns sinkOp.cacheMode = cacheMode sinkOp.hivePartitionKey = hivePartitionKey - sinkOp.useUnionRDD = useUnionRDD + sinkOp.isInsertInto = isInsertInto _createAndSetParents(sinkOp, hiveTerminalOp.getParentOperators).asInstanceOf[TerminalOperator] } diff --git a/src/main/scala/shark/execution/TableReader.scala b/src/main/scala/shark/execution/TableReader.scala index 85a6b913..b39f12b3 100644 --- a/src/main/scala/shark/execution/TableReader.scala +++ b/src/main/scala/shark/execution/TableReader.scala @@ -40,42 +40,30 @@ import shark.tachyon.TachyonException */ trait TableReader extends LogHelper { - def makeRDDForTable(hiveTable: HiveTable): RDD[_] - - def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] - -} - -trait MemoryTableReader extends TableReader { - type PruningFunctionType = (RDD[_], collection.Map[Int, TablePartitionStats]) => RDD[_] def makeRDDForTable( hiveTable: HiveTable, - pruningFnOpt: Option[PruningFunctionType] + pruningFnOpt: Option[PruningFunctionType] = None ): RDD[_] def makeRDDForPartitionedTable( partitions: Seq[HivePartition], - pruningFnOpt: Option[PruningFunctionType] + pruningFnOpt: Option[PruningFunctionType] = None ): RDD[_] } /** Helper class for scanning tables stored in Tachyon. */ -class TachyonTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReader { +class TachyonTableReader(@transient _tableDesc: TableDesc) extends TableReader { // Split from 'databaseName.tableName' private val _tableNameSplit = _tableDesc.getTableName.split('.') private val _databaseName = _tableNameSplit(0) private val _tableName = _tableNameSplit(1) - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { - makeRDDForTable(hiveTable, pruningFnOpt = None) - } - override def makeRDDForTable( hiveTable: HiveTable, - pruningFnOpt: Option[PruningFunctionType] + pruningFnOpt: Option[PruningFunctionType] = None ): RDD[_] = { // Table is in Tachyon. val tableKey = SharkEnv.makeTachyonTableKey(_databaseName, _tableName) @@ -86,22 +74,13 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends MemoryTableRe val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( statsByteBuffer.array()) logInfo("Fetching table " + tableKey + " stats from Tachyon.") - val (tableRdd, tableStats) = (SharkEnv.tachyonUtil.createRDD(tableKey), indexToStats) - if (pruningFnOpt.isDefined) { - val pruningFn = pruningFnOpt.get - pruningFn(tableRdd, tableStats) - } else { - tableRdd - } - } - - override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { - throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") + val tableRdd = SharkEnv.tachyonUtil.createRDD(tableKey) + pruningFnOpt.map(_(tableRdd, indexToStats)).getOrElse(tableRdd) } override def makeRDDForPartitionedTable( partitions: Seq[HivePartition], - pruningFnOpt: Option[PruningFunctionType] + pruningFnOpt: Option[PruningFunctionType] = None ): RDD[_] = { throw new UnsupportedOperationException("Partitioned tables are not yet supported for Tachyon.") } @@ -109,7 +88,7 @@ class TachyonTableReader(@transient _tableDesc: TableDesc) extends MemoryTableRe } /** Helper class for scanning tables stored in Spark's block manager */ -class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReader { +class HeapTableReader(@transient _tableDesc: TableDesc) extends TableReader { // Split from 'databaseName.tableName' private val _tableNameSplit = _tableDesc.getTableName.split('.') @@ -119,7 +98,7 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade /** Fetches and optionally prunes the RDD for `_tableName` from the Shark metastore. */ override def makeRDDForTable( hiveTable: HiveTable, - pruningFnOpt: Option[PruningFunctionType] + pruningFnOpt: Option[PruningFunctionType] = None ): RDD[_] = { logInfo("Loading table %s.%s from Spark block manager".format(_databaseName, _tableName)) val tableOpt = SharkEnv.memoryMetadataManager.getMemoryTable(_databaseName, _tableName) @@ -128,18 +107,10 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } val table = tableOpt.get - val (tableRdd, tableStats) = (table.getRDD.get, table.getStats.get) + val tableRdd = table.getRDD.get + val tableStats = table.getStats.get // Prune if an applicable function is given. - if (pruningFnOpt.isDefined) { - val pruningFn = pruningFnOpt.get - pruningFn(tableRdd, tableStats) - } else { - tableRdd - } - } - - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = { - makeRDDForTable(hiveTable, pruningFnOpt = None) + pruningFnOpt.map(_(tableRdd, tableStats)).getOrElse(tableRdd) } /** @@ -151,7 +122,7 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade */ override def makeRDDForPartitionedTable( partitions: Seq[HivePartition], - pruningFnOpt: Option[PruningFunctionType] + pruningFnOpt: Option[PruningFunctionType] = None ): RDD[_] = { val hivePartitionRDDs = partitions.map { partition => val partDesc = Utilities.getPartitionDesc(partition) @@ -182,12 +153,8 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade throwMissingPartitionException(partitionKeyStr) } val (hivePartitionRDD, hivePartitionStats) = (rddAndStatsOpt.get._1, rddAndStatsOpt.get._2) - val prunedPartitionRDD = if (pruningFnOpt.isDefined) { - val pruningFn = pruningFnOpt.get - pruningFn(hivePartitionRDD, hivePartitionStats) - } else { - hivePartitionRDD - } + val prunedPartitionRDD = pruningFnOpt.map(_(hivePartitionRDD, hivePartitionStats)) + .getOrElse(hivePartitionRDD) prunedPartitionRDD.mapPartitions { iter => if (iter.hasNext) { // Map each tuple to a row object @@ -209,10 +176,6 @@ class HeapTableReader(@transient _tableDesc: TableDesc) extends MemoryTableReade } } - override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { - makeRDDForPartitionedTable(partitions, pruningFnOpt = None) - } - /** * Thrown if the table identified by the (_databaseName, _tableName) pair cannot be found in * the Shark metastore. diff --git a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala index cdbe8ba3..d45872b1 100755 --- a/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala +++ b/src/main/scala/shark/parse/SharkSemanticAnalyzer.scala @@ -188,8 +188,8 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with databaseName, cachedTableName).get // INSERT INTO or OVERWRITE update on a cached table. qb.targetTableDesc = tableDesc - // If useUnionRDD is true, the sink op is for INSERT INTO. - val useUnionRDD = qbParseInfo.isInsertIntoTable(cachedTableName) + // If `isInsertInto` is true, the sink op is for INSERT INTO. + val isInsertInto = qbParseInfo.isInsertIntoTable(cachedTableName) val isPartitioned = SharkEnv.memoryMetadataManager.isHivePartitioned( databaseName, cachedTableName) var hivePartitionKey = if (isPartitioned) { @@ -216,7 +216,7 @@ class SharkSemanticAnalyzer(conf: HiveConf) extends SemanticAnalyzer(conf) with _resSchema.size, /* numColumns */ hivePartitionKey, table.cacheMode, - useUnionRDD) + isInsertInto) } } else { throw new SemanticException( From b246a2c788542d52e03eff07fe25b698349dc800 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 15 Jan 2014 18:17:42 -0800 Subject: [PATCH 319/331] Removed TGF related compilation warnings. --- .../tablerdd/TableRDDGenerated_generator.py | 4 +- src/main/scala/shark/api/RDDTable.scala | 2 +- .../scala/shark/api/TableRDDGenerated.scala | 94 +++++++++---------- src/main/scala/shark/tgf/TGF.scala | 1 + 4 files changed, 51 insertions(+), 50 deletions(-) diff --git a/src/main/resources/tablerdd/TableRDDGenerated_generator.py b/src/main/resources/tablerdd/TableRDDGenerated_generator.py index 0e4626d1..45deec03 100755 --- a/src/main/resources/tablerdd/TableRDDGenerated_generator.py +++ b/src/main/resources/tablerdd/TableRDDGenerated_generator.py @@ -31,7 +31,7 @@ package shark.api // *** This file is auto-generated from TableRDDGenerated_generator.py *** - +import scala.language.implicitConversions import org.apache.spark.rdd.RDD import org.apache.spark.{TaskContext, Partition} @@ -78,7 +78,7 @@ def schema = prev.schema override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple$num[$list]] = { prev.compute(split, context).map( row => new Tuple$num[$list]( diff --git a/src/main/scala/shark/api/RDDTable.scala b/src/main/scala/shark/api/RDDTable.scala index fa3429a1..c0496e9e 100644 --- a/src/main/scala/shark/api/RDDTable.scala +++ b/src/main/scala/shark/api/RDDTable.scala @@ -18,7 +18,7 @@ package shark.api // *** This file is auto-generated from RDDTable_generator.py *** - +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.rdd.RDD diff --git a/src/main/scala/shark/api/TableRDDGenerated.scala b/src/main/scala/shark/api/TableRDDGenerated.scala index 0980630c..a0189831 100644 --- a/src/main/scala/shark/api/TableRDDGenerated.scala +++ b/src/main/scala/shark/api/TableRDDGenerated.scala @@ -21,7 +21,7 @@ package shark.api // *** This file is auto-generated from TableRDDGenerated_generator.py *** - +import scala.language.implicitConversions import org.apache.spark.rdd.RDD import org.apache.spark.{TaskContext, Partition} @@ -55,7 +55,7 @@ class TableRDD1[T1](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple1[T1]] = { prev.compute(split, context).map( row => new Tuple1[T1]( @@ -78,7 +78,7 @@ class TableRDD2[T1, T2](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple2[T1, T2]] = { prev.compute(split, context).map( row => new Tuple2[T1, T2]( @@ -101,7 +101,7 @@ class TableRDD3[T1, T2, T3](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple3[T1, T2, T3]] = { prev.compute(split, context).map( row => new Tuple3[T1, T2, T3]( @@ -125,7 +125,7 @@ class TableRDD4[T1, T2, T3, T4](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple4[T1, T2, T3, T4]] = { prev.compute(split, context).map( row => new Tuple4[T1, T2, T3, T4]( @@ -149,7 +149,7 @@ class TableRDD5[T1, T2, T3, T4, T5](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple5[T1, T2, T3, T4, T5]] = { prev.compute(split, context).map( row => new Tuple5[T1, T2, T3, T4, T5]( @@ -173,7 +173,7 @@ class TableRDD6[T1, T2, T3, T4, T5, T6](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple6[T1, T2, T3, T4, T5, T6]] = { prev.compute(split, context).map( row => new Tuple6[T1, T2, T3, T4, T5, T6]( @@ -198,7 +198,7 @@ class TableRDD7[T1, T2, T3, T4, T5, T6, T7](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple7[T1, T2, T3, T4, T5, T6, T7]] = { prev.compute(split, context).map( row => new Tuple7[T1, T2, T3, T4, T5, T6, T7]( @@ -223,7 +223,7 @@ class TableRDD8[T1, T2, T3, T4, T5, T6, T7, T8](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]] = { prev.compute(split, context).map( row => new Tuple8[T1, T2, T3, T4, T5, T6, T7, T8]( @@ -248,7 +248,7 @@ class TableRDD9[T1, T2, T3, T4, T5, T6, T7, T8, T9](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]] = { prev.compute(split, context).map( row => new Tuple9[T1, T2, T3, T4, T5, T6, T7, T8, T9]( @@ -274,7 +274,7 @@ class TableRDD10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]] = { prev.compute(split, context).map( row => new Tuple10[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10]( @@ -300,7 +300,7 @@ class TableRDD11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11](prev: TableRDD, override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]] = { prev.compute(split, context).map( row => new Tuple11[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11]( @@ -326,7 +326,7 @@ class TableRDD12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12](prev: TableR override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]] = { prev.compute(split, context).map( row => new Tuple12[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12]( @@ -353,7 +353,7 @@ class TableRDD13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13](prev: T override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]] = { prev.compute(split, context).map( row => new Tuple13[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13]( @@ -380,7 +380,7 @@ class TableRDD14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14](pr override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]] = { prev.compute(split, context).map( row => new Tuple14[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14]( @@ -407,7 +407,7 @@ class TableRDD15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]] = { prev.compute(split, context).map( row => new Tuple15[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15]( @@ -435,7 +435,7 @@ class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): + override def compute(split: Partition, context: TaskContext): Iterator[Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]] = { prev.compute(split, context).map( row => new Tuple16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16]( @@ -449,10 +449,10 @@ class TableRDD16[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, +class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17](prev: TableRDD, tags: Seq[ClassTag[_]]) - extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + extends RDD[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]](prev) { def schema = prev.schema @@ -465,11 +465,11 @@ class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]] = { prev.compute(split, context).map( row => - new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + new Tuple17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17]( row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), @@ -481,10 +481,10 @@ class TableRDD17[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, +class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18](prev: TableRDD, tags: Seq[ClassTag[_]]) - extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + extends RDD[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]](prev) { def schema = prev.schema @@ -497,11 +497,11 @@ class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]] = { prev.compute(split, context).map( row => - new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + new Tuple18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18]( row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), @@ -514,10 +514,10 @@ class TableRDD18[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, +class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19](prev: TableRDD, tags: Seq[ClassTag[_]]) - extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + extends RDD[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19]](prev) { def schema = prev.schema @@ -530,11 +530,11 @@ class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19]] = { prev.compute(split, context).map( row => - new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + new Tuple19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19]( row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), @@ -547,10 +547,10 @@ class TableRDD19[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, +class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20](prev: TableRDD, tags: Seq[ClassTag[_]]) - extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + extends RDD[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20]](prev) { def schema = prev.schema @@ -563,11 +563,11 @@ class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20]] = { prev.compute(split, context).map( row => - new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + new Tuple20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20]( row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), @@ -580,10 +580,10 @@ class TableRDD20[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, +class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21](prev: TableRDD, tags: Seq[ClassTag[_]]) - extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + extends RDD[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21]](prev) { def schema = prev.schema @@ -596,11 +596,11 @@ class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21]] = { prev.compute(split, context).map( row => - new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + new Tuple21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21]( row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), @@ -614,10 +614,10 @@ class TableRDD21[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 } } -class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, +class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22](prev: TableRDD, tags: Seq[ClassTag[_]]) - extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + extends RDD[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22]](prev) { def schema = prev.schema @@ -630,11 +630,11 @@ class TableRDD22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T1 override def getPartitions = prev.getPartitions - override def compute(split: Partition, context: TaskContext): - Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + override def compute(split: Partition, context: TaskContext): + Iterator[Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22]] = { prev.compute(split, context).map( row => - new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, + new Tuple22[T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22]( row.getPrimitiveGeneric[T1](0), row.getPrimitiveGeneric[T2](1), row.getPrimitiveGeneric[T3](2), row.getPrimitiveGeneric[T4](3), row.getPrimitiveGeneric[T5](4), row.getPrimitiveGeneric[T6](5), diff --git a/src/main/scala/shark/tgf/TGF.scala b/src/main/scala/shark/tgf/TGF.scala index cc3aceaa..b57d4053 100644 --- a/src/main/scala/shark/tgf/TGF.scala +++ b/src/main/scala/shark/tgf/TGF.scala @@ -20,6 +20,7 @@ package shark.tgf import java.sql.Timestamp import java.util.Date +import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import scala.util.parsing.combinator._ From 6f8ebcbfeb5d0a83a4699596f2272a007cca6555 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 15 Jan 2014 18:36:24 -0800 Subject: [PATCH 320/331] Removed more compiler warning. --- src/main/resources/tablerdd/rddtable_generator.py | 1 + src/test/scala/shark/memstore2/ColumnStatsSuite.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/src/main/resources/tablerdd/rddtable_generator.py b/src/main/resources/tablerdd/rddtable_generator.py index f5c2e291..eda23d05 100755 --- a/src/main/resources/tablerdd/rddtable_generator.py +++ b/src/main/resources/tablerdd/rddtable_generator.py @@ -46,6 +46,7 @@ def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, ind // *** This file is auto-generated from RDDTable_generator.py *** +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.rdd.RDD diff --git a/src/test/scala/shark/memstore2/ColumnStatsSuite.scala b/src/test/scala/shark/memstore2/ColumnStatsSuite.scala index 31968103..2da1959c 100644 --- a/src/test/scala/shark/memstore2/ColumnStatsSuite.scala +++ b/src/test/scala/shark/memstore2/ColumnStatsSuite.scala @@ -18,6 +18,7 @@ package shark.memstore2 import java.sql.Timestamp +import scala.language.implicitConversions import org.apache.hadoop.io.Text From 3be92ce8841c5c51921258f76a28e06b68dedb7d Mon Sep 17 00:00:00 2001 From: Harvey Date: Wed, 15 Jan 2014 19:27:52 -0800 Subject: [PATCH 321/331] Correctly detect Tachyon connections. Don't run TachyonSQLSuite if Tachyon isn't running... --- .../scala/shark/tachyon/TachyonUtilImpl.scala | 3 ++- src/test/scala/shark/TachyonSQLSuite.scala | 12 +++++++++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala b/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala index 3ee76593..8e4eab8d 100644 --- a/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala +++ b/src/tachyon_enabled/scala/shark/tachyon/TachyonUtilImpl.scala @@ -87,7 +87,8 @@ class TachyonUtilImpl( isTachyonTableRdd } - override def tachyonEnabled(): Boolean = (master != null && warehousePath != null) + override def tachyonEnabled(): Boolean = + (master != null && warehousePath != null && client.isConnected) override def tableExists(tableKey: String, hivePartitionKeyOpt: Option[String]): Boolean = { client.exist(getPath(tableKey, hivePartitionKeyOpt)) diff --git a/src/test/scala/shark/TachyonSQLSuite.scala b/src/test/scala/shark/TachyonSQLSuite.scala index 4593516b..899bc1d4 100644 --- a/src/test/scala/shark/TachyonSQLSuite.scala +++ b/src/test/scala/shark/TachyonSQLSuite.scala @@ -46,10 +46,17 @@ class TachyonSQLSuite extends FunSuite with BeforeAndAfterAll { // Determine if Tachyon enabled at runtime. val isTachyonEnabled = SharkEnv.tachyonUtil.tachyonEnabled() - sc.runSql("create table test_tachyon as select * from test") + + override def beforeAll() { + if (isTachyonEnabled) { + sc.runSql("create table test_tachyon as select * from test") + } + } override def afterAll() { - sc.runSql("drop table test_tachyon") + if (isTachyonEnabled) { + sc.runSql("drop table test_tachyon") + } } private def isTachyonTable( @@ -77,7 +84,6 @@ class TachyonSQLSuite extends FunSuite with BeforeAndAfterAll { } if (isTachyonEnabled) { - ////////////////////////////////////////////////////////////////////////////// // basic SQL ////////////////////////////////////////////////////////////////////////////// From 6d634b522acdca5413543e735cea48f9a9e9bb5d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 15 Jan 2014 21:49:59 -0800 Subject: [PATCH 322/331] Add a JavaSharkContext to SharkRunner and use it in JavaAPISuite --- src/test/java/shark/JavaAPISuite.java | 4 +++- src/test/scala/shark/SharkRunner.scala | 12 +++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/test/java/shark/JavaAPISuite.java b/src/test/java/shark/JavaAPISuite.java index 01f6fe58..0f05d0fc 100644 --- a/src/test/java/shark/JavaAPISuite.java +++ b/src/test/java/shark/JavaAPISuite.java @@ -48,7 +48,9 @@ public static void oneTimeSetUp() { // Intentionally leaving this here since SBT doesn't seem to display junit tests well ... System.out.println("running JavaAPISuite ================================================"); - sc = SharkEnv.initWithJavaSharkContext("JavaAPISuite", "local"); + // Check if the SharkEnv's SharkContext has already been initialized. If so, use that to + // instantiate a JavaSharkContext. + sc = SharkRunner.initWithJava(); sc.sql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true"); diff --git a/src/test/scala/shark/SharkRunner.scala b/src/test/scala/shark/SharkRunner.scala index 754737b9..573ecec2 100644 --- a/src/test/scala/shark/SharkRunner.scala +++ b/src/test/scala/shark/SharkRunner.scala @@ -19,6 +19,7 @@ package shark import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME +import shark.api.JavaSharkContext import shark.memstore2.MemoryMetadataManager @@ -29,8 +30,8 @@ object SharkRunner { val MASTER = "local" var sc: SharkContext = _ - var sharkMetastore: MemoryMetadataManager = _ + var javaSc: JavaSharkContext = _ def init(): SharkContext = synchronized { if (sc == null) { @@ -41,8 +42,6 @@ object SharkRunner { sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) - sharkMetastore = SharkEnv.memoryMetadataManager - // second db sc.sql("create database if not exists seconddb") @@ -51,6 +50,13 @@ object SharkRunner { sc } + def initWithJava(): JavaSharkContext = synchronized { + if (javaSc == null) { + javaSc = new JavaSharkContext(init()) + } + javaSc + } + /** * Tables accessible by any test. Their properties should remain constant across * tests. From c73ff45986efe9790cdb7f76baf531441b87c79c Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 16 Jan 2014 14:37:28 -0800 Subject: [PATCH 323/331] Remove duplicate RDDTable_generator.py --- .../resources/tablerdd/RDDTable_generator.py | 98 ------------------- 1 file changed, 98 deletions(-) delete mode 100755 src/main/resources/tablerdd/RDDTable_generator.py diff --git a/src/main/resources/tablerdd/RDDTable_generator.py b/src/main/resources/tablerdd/RDDTable_generator.py deleted file mode 100755 index 82c15371..00000000 --- a/src/main/resources/tablerdd/RDDTable_generator.py +++ /dev/null @@ -1,98 +0,0 @@ -#!/usr/bin/python -from string import Template -import sys -from generator_utils import * - -## This script generates RDDtable.scala - -p = sys.stdout - -# e.g. createList(1,3, "T[", "]", ",") gives T[1],T[2],T[3] -def createList(start, stop, prefix, suffix="", sep = ",", newlineAfter = 70, indent = 0): - res = "" - oneLine = res - for y in range(start,stop+1): - res += prefix + str(y) + suffix - oneLine += prefix + str(y) + suffix - if y != stop: - res += sep - oneLine += sep - if len(oneLine) > newlineAfter: - res += "\n" + " "*indent - oneLine = "" - return res - -### The SparkContext declaration - -prefix = """ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.api - -// *** This file is auto-generated from RDDTable_generator.py *** - -import scala.language.implicitConversions - -import org.apache.spark.rdd.RDD -import scala.reflect.ClassTag - -object RDDTableImplicits { - private type C[T] = ClassTag[T] - -""" - -p.write(prefix) - -for x in range(2,23): - - tableClass = Template( -""" - implicit def rddToTable$num[$tmlist] - (rdd: RDD[($tlist)]): RDDTableFunctions = RDDTable(rdd) - -""").substitute(num = x, tmlist = createList(1, x, "T", ": C", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4)) - p.write(tableClass) - -prefix = """ -} - -object RDDTable { - - private type C[T] = ClassTag[T] - private def ct[T](implicit c : ClassTag[T]) = c -""" - -p.write(prefix) - -for x in range(2,23): - - tableClass = Template( -""" - def apply[$tmlist] - (rdd: RDD[($tlist)]) = { - val classTag = implicitly[ClassTag[Seq[Any]]] - val rddSeq: RDD[Seq[_]] = rdd.map(t => t.productIterator.toList.asInstanceOf[Seq[Any]])(classTag) - new RDDTableFunctions(rddSeq, Seq($mtlist)) - } - -""").substitute(tmlist = createList(1, x, "T", ": C", ", ", indent=4), tlist = createList(1, x, "T", "", ", ", indent=4), - mtlist = createList(1, x, "ct[T", "]", ", ", indent=4)) - p.write(tableClass) - - -p.write("}\n") From 003e7ed608fedfe3ff47731e023e4105a1106b9d Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Mon, 20 Jan 2014 18:49:42 -0800 Subject: [PATCH 324/331] Use lazy SparkConf in KryoSerializer during init. Guarantees that master and slaves have their own SparkConf references. --- src/main/scala/shark/SharkEnv.scala | 2 -- .../execution/serialization/KryoSerializer.scala | 13 ++++--------- .../serialization/SerializationSuite.scala | 3 --- 3 files changed, 4 insertions(+), 14 deletions(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 65d56bd7..0bd2ed35 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -38,7 +38,6 @@ object SharkEnv extends LogHelper { val master = System.getenv("MASTER") initWithSharkContext(jobName, master) } - KryoSerializer.initWithSharkContext(sc) sc } @@ -57,7 +56,6 @@ object SharkEnv extends LogHelper { Nil, executorEnvVars) sc.addSparkListener(new StatsReportListener()) - KryoSerializer.initWithSharkContext(sc) sc } diff --git a/src/main/scala/shark/execution/serialization/KryoSerializer.scala b/src/main/scala/shark/execution/serialization/KryoSerializer.scala index a5c39148..0532fbcc 100644 --- a/src/main/scala/shark/execution/serialization/KryoSerializer.scala +++ b/src/main/scala/shark/execution/serialization/KryoSerializer.scala @@ -19,7 +19,7 @@ package shark.execution.serialization import java.nio.ByteBuffer -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.serializer.{KryoSerializer => SparkKryoSerializer} import shark.SharkContext @@ -31,14 +31,9 @@ import shark.SharkContext */ object KryoSerializer { - @transient var ser: SparkKryoSerializer = _ - - def initWithSharkContext(sc: SharkContext) { - ser = new SparkKryoSerializer(sc.sparkEnv.conf) - } - - def initWithSparkConf(sparkConf: SparkConf) { - ser = new SparkKryoSerializer(sparkConf) + @transient lazy val ser: SparkKryoSerializer = { + val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + new SparkKryoSerializer(sparkConf) } def serialize[T](o: T): Array[Byte] = { diff --git a/src/test/scala/shark/execution/serialization/SerializationSuite.scala b/src/test/scala/shark/execution/serialization/SerializationSuite.scala index cc2845c2..43c97a0c 100755 --- a/src/test/scala/shark/execution/serialization/SerializationSuite.scala +++ b/src/test/scala/shark/execution/serialization/SerializationSuite.scala @@ -46,9 +46,6 @@ object SerializationSuite { class SerializationSuite extends FunSuite { - // Initialize the Shark KryoSerializer singleton. - KryoSerializer.initWithSparkConf(new SparkConf(loadDefaults = false)) - test("Java serializing object inspectors") { val oi = PrimitiveObjectInspectorFactory.javaStringObjectInspector From 5ba0ca03758232f2fe8739c73f3dd5d5d6dce50e Mon Sep 17 00:00:00 2001 From: Harvey Date: Wed, 22 Jan 2014 14:44:08 -0800 Subject: [PATCH 325/331] Add SparkConf constructor in SharkContext. --- src/main/scala/shark/SharkContext.scala | 32 +++++++++++++++++-------- src/main/scala/shark/SharkEnv.scala | 10 ++++++-- 2 files changed, 30 insertions(+), 12 deletions(-) diff --git a/src/main/scala/shark/SharkContext.scala b/src/main/scala/shark/SharkContext.scala index 9dff57f7..b20847f3 100755 --- a/src/main/scala/shark/SharkContext.scala +++ b/src/main/scala/shark/SharkContext.scala @@ -31,7 +31,8 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.spark.{SparkContext, SparkEnv} +import org.apache.spark.{SparkConf, SparkContext, SparkEnv} +import org.apache.spark.scheduler.SplitInfo import org.apache.spark.rdd.RDD import shark.api._ @@ -39,21 +40,32 @@ import shark.tgf.TGF class SharkContext( - master: String, - jobName: String, - sparkHome: String, - jars: Seq[String], - environment: Map[String, String]) - extends SparkContext(master, jobName, sparkHome, jars, environment) { + config: SparkConf, + preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) + extends SparkContext(config, preferredNodeLocationData) { + import SharkContext._ @transient val sparkEnv = SparkEnv.get - SharkContext.init() - import SharkContext._ - private type C[T] = ClassTag[T] private def ct[T](implicit c : ClassTag[T]) = c + SharkContext.init() + + def this( + master: String, + jobName: String, + sparkHome: String, + jars: Seq[String], + environment: Map[String, String]) { + this( + (new SparkConf()) + .setMaster(master) + .setAppName(jobName) + .setSparkHome(sparkHome) + .setJars(jars) + .setExecutorEnv(environment.toSeq)) + } /** * Execute the command and return the results as a sequence. Each element diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 0bd2ed35..4523d803 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -19,7 +19,7 @@ package shark import scala.collection.mutable.{HashMap, HashSet} -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.StatsReportListener @@ -59,10 +59,16 @@ object SharkEnv extends LogHelper { sc } + def initWithSharkContext(conf: SparkConf): SharkContext = { + conf.setExecutorEnv(executorEnvVars.toSeq) + initWithSharkContext(new SharkContext(conf)) + } + def initWithSharkContext(newSc: SharkContext): SharkContext = { if (sc != null) { sc.stop() } + sc.addSparkListener(new StatsReportListener()) sc = newSc sc } @@ -92,7 +98,7 @@ object SharkEnv extends LogHelper { executorEnvVars.put("TACHYON_WAREHOUSE_PATH", getEnv("TACHYON_WAREHOUSE_PATH")) val activeSessions = new HashSet[String] - + var sc: SharkContext = _ val shuffleSerializerName = classOf[ShuffleSerializer].getName From c950362e5794d896dc24a5fa7cb8df17df07b992 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Wed, 22 Jan 2014 15:51:41 -0800 Subject: [PATCH 326/331] Use Spark 0.9.0 incubating --- project/SharkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 420abd77..5ed67c4b 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -28,7 +28,7 @@ object SharkBuild extends Build { // Shark version val SHARK_VERSION = "0.9.0-SNAPSHOT" - val SPARK_VERSION = "0.9.0-incubating-SNAPSHOT" + val SPARK_VERSION = "0.9.0-incubating" val SCALA_VERSION = "2.10.3" From 149716b8ea43093099b4b14d4a27c64ad2da21e1 Mon Sep 17 00:00:00 2001 From: Harvey Date: Wed, 22 Jan 2014 16:00:06 -0800 Subject: [PATCH 327/331] Fix NPE in SharkEnv --- src/main/scala/shark/SharkEnv.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark/SharkEnv.scala b/src/main/scala/shark/SharkEnv.scala index 4523d803..6060ef51 100755 --- a/src/main/scala/shark/SharkEnv.scala +++ b/src/main/scala/shark/SharkEnv.scala @@ -68,8 +68,8 @@ object SharkEnv extends LogHelper { if (sc != null) { sc.stop() } - sc.addSparkListener(new StatsReportListener()) sc = newSc + sc.addSparkListener(new StatsReportListener()) sc } From 5bb4999d6c2a51e662d44db6651ab95b37e24243 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Sun, 26 Jan 2014 23:34:54 +0000 Subject: [PATCH 328/331] Change vers # for this unofficial release branch that is "post scala 2.10" but "pre hive 10.0". --- project/SharkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 5ed67c4b..95b8f220 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -26,7 +26,7 @@ import scala.util.Properties.{ envOrNone => env } object SharkBuild extends Build { // Shark version - val SHARK_VERSION = "0.9.0-SNAPSHOT" + val SHARK_VERSION = "0.9.0-hive0.9-SNAPSHOT" val SPARK_VERSION = "0.9.0-incubating" From 85f9c4c9518e7e5be99b13596e76b88005b109d2 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 24 Jan 2014 01:17:47 -0800 Subject: [PATCH 329/331] Merge pull request #249 from kayousterhout/cache_generosity Accept spaces before (UN)CACHE, more helpful error message. --- src/main/scala/shark/SharkDriver.scala | 2 +- src/main/scala/shark/util/QueryRewriteUtils.scala | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/SharkDriver.scala b/src/main/scala/shark/SharkDriver.scala index 1dd8446f..c227e93d 100755 --- a/src/main/scala/shark/SharkDriver.scala +++ b/src/main/scala/shark/SharkDriver.scala @@ -183,7 +183,7 @@ private[shark] class SharkDriver(conf: HiveConf) extends Driver(conf) with LogHe try { val command = { - val varSubbedCmd = new VariableSubstitution().substitute(conf, cmd) + val varSubbedCmd = new VariableSubstitution().substitute(conf, cmd).trim val cmdInUpperCase = varSubbedCmd.toUpperCase if (cmdInUpperCase.startsWith("CACHE")) { QueryRewriteUtils.cacheToAlterTable(varSubbedCmd) diff --git a/src/main/scala/shark/util/QueryRewriteUtils.scala b/src/main/scala/shark/util/QueryRewriteUtils.scala index 8ddd6e4a..8d44f8a8 100644 --- a/src/main/scala/shark/util/QueryRewriteUtils.scala +++ b/src/main/scala/shark/util/QueryRewriteUtils.scala @@ -30,7 +30,8 @@ object QueryRewriteUtils { val tableName = cmdSplit(1) "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache' = 'true')".format(tableName) } else { - throw new SemanticException("CACHE accepts a single table name: 'CACHE
'") + throw new SemanticException( + s"CACHE accepts a single table name: 'CACHE
' (received command: '$cmd')") } } @@ -40,7 +41,8 @@ object QueryRewriteUtils { val tableName = cmdSplit(1) "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache' = 'false')".format(tableName) } else { - throw new SemanticException("UNCACHE accepts a single table name: 'UNCACHE
'") + throw new SemanticException( + s"UNCACHE accepts a single table name: 'UNCACHE
' (received command: '$cmd')") } } } From 80021b510534db3a5242a44159f16640a42e1e90 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Fri, 24 Jan 2014 01:19:05 -0800 Subject: [PATCH 330/331] Merge pull request #251 from kayousterhout/reload_log Add console logging about reloaded RDDs. --- src/main/scala/shark/SharkCliDriver.scala | 8 ++++++-- src/main/scala/shark/memstore2/TableRecovery.scala | 12 ++++++++++-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 32565865..5a36458f 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -49,6 +49,7 @@ import org.apache.thrift.transport.TSocket import shark.memstore2.TableRecovery object SharkCliDriver { + val SKIP_RDD_RELOAD_FLAG = "-skipRddReload" private var prompt = "shark" private var prompt2 = " " // when ';' is not yet seen. @@ -78,7 +79,7 @@ object SharkCliDriver { } def main(args: Array[String]) { - val hiveArgs = args.filterNot(_.equals("-skipRddReload")) + val hiveArgs = args.filterNot(_.equals(SKIP_RDD_RELOAD_FLAG)) val reloadRdds = hiveArgs.length == args.length val oproc = new OptionsProcessor() if (!oproc.process_stage1(hiveArgs)) { @@ -266,7 +267,10 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe if (!ss.isRemoteMode()) { SharkEnv.init() if (reloadRdds) { - TableRecovery.reloadRdds(processCmd(_)) + console.printInfo( + "Reloading cached RDDs from previous Shark sessions... (use %s flag to skip reloading)" + .format(SharkCliDriver.SKIP_RDD_RELOAD_FLAG)) + TableRecovery.reloadRdds(processCmd(_), Some(console)) } } diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala index 21fbddb7..adf61061 100644 --- a/src/main/scala/shark/memstore2/TableRecovery.scala +++ b/src/main/scala/shark/memstore2/TableRecovery.scala @@ -22,6 +22,7 @@ import java.util.{HashMap => JavaHashMap} import scala.collection.JavaConversions.asScalaBuffer import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.session.SessionState import shark.{LogHelper, SharkEnv} import shark.util.QueryRewriteUtils @@ -39,8 +40,10 @@ object TableRecovery extends LogHelper { * a) Creating the table metadata in Hive Meta Store * b) Loading the table as an RDD in memory * @see SharkServer for an example usage. + * @param console Optional SessionState.LogHelper used, if present, to log information about + the tables that get reloaded. */ - def reloadRdds(cmdRunner: String => Unit) { + def reloadRdds(cmdRunner: String => Unit, console: Option[SessionState.LogHelper] = None) { // Filter for tables that should be reloaded into the cache. val currentDbName = db.getCurrentDatabase() for (databaseName <- db.getAllDatabases(); tableName <- db.getAllTables(databaseName)) { @@ -48,7 +51,12 @@ object TableRecovery extends LogHelper { val tblProps = hiveTable.getParameters val cacheMode = CacheType.fromString(tblProps.get(SharkTblProperties.CACHE_FLAG.varname)) if (cacheMode == CacheType.MEMORY) { - logInfo("Reloading %s.%s into memory.".format(databaseName, tableName)) + val logMessage = "Reloading %s.%s into memory.".format(databaseName, tableName) + if (console.isDefined) { + console.get.printInfo(logMessage) + } else { + logInfo(logMessage) + } val cmd = QueryRewriteUtils.cacheToAlterTable("CACHE %s".format(tableName)) cmdRunner(cmd) } From 314a90f2833593b7a7f11f10cfcbb294010f81a0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 27 Jan 2014 19:34:05 -0800 Subject: [PATCH 331/331] Merge pull request #253 from harveyfeng/tablescan Correctly detect in-memory RDDs for table scans. --- .../shark/execution/TableScanOperator.scala | 19 ++++++++++--------- src/test/java/shark/JavaAPISuite.java | 6 ------ src/test/scala/shark/SQLSuite.scala | 4 ++++ 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/main/scala/shark/execution/TableScanOperator.scala b/src/main/scala/shark/execution/TableScanOperator.scala index c5684ca2..eaba7e9b 100755 --- a/src/main/scala/shark/execution/TableScanOperator.scala +++ b/src/main/scala/shark/execution/TableScanOperator.scala @@ -37,7 +37,9 @@ import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import shark.{LogHelper, SharkConfVars, SharkEnv} import shark.execution.optimization.ColumnPruner -import shark.memstore2.{CacheType, ColumnarSerDe, MemoryMetadataManager} +import shark.memstore2.CacheType +import shark.memstore2.CacheType._ +import shark.memstore2.{ColumnarSerDe, MemoryMetadataManager} import shark.memstore2.{TablePartition, TablePartitionStats} import shark.util.HiveUtils @@ -70,22 +72,25 @@ class TableScanOperator extends TopOperator[TableScanDesc] { @BeanProperty var tableDesc: TableDesc = _ + // True if table data is stored the Spark heap. @BeanProperty var isInMemoryTableScan: Boolean = _ + @BeanProperty var cacheMode: CacheType.CacheType = _ + override def initializeOnMaster() { // Create a local copy of the HiveConf that will be assigned job properties and, for disk reads, // broadcasted to slaves. localHConf = new HiveConf(super.hconf) + cacheMode = CacheType.fromString( + tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) isInMemoryTableScan = SharkEnv.memoryMetadataManager.containsTable( table.getDbName, table.getTableName) } override def outputObjectInspector() = { - val cacheMode = CacheType.fromString( - tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) if (parts == null) { - val serializer = if (CacheType.shouldCache(cacheMode)) { + val serializer = if (isInMemoryTableScan || cacheMode == CacheType.TACHYON) { new ColumnarSerDe } else { tableDesc.getDeserializerClass().newInstance() @@ -94,7 +99,7 @@ class TableScanOperator extends TopOperator[TableScanDesc] { serializer.getObjectInspector() } else { val partProps = firstConfPartDesc.getProperties() - val partSerDe = if (CacheType.shouldCache(cacheMode)) { + val partSerDe = if (isInMemoryTableScan || cacheMode == CacheType.TACHYON) { new ColumnarSerDe } else { firstConfPartDesc.getDeserializerClass().newInstance() @@ -115,8 +120,6 @@ class TableScanOperator extends TopOperator[TableScanDesc] { // 1. Spark heap (block manager), accessed through the Shark MemoryMetadataManager // 2. Tachyon table // 3. Hive table on HDFS (or other Hadoop storage) - val cacheMode = CacheType.fromString( - tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) // TODO(harvey): Pruning Hive-partitioned, cached tables isn't supported yet. if (isInMemoryTableScan || cacheMode == CacheType.TACHYON) { if (isInMemoryTableScan) { @@ -147,8 +150,6 @@ class TableScanOperator extends TopOperator[TableScanDesc] { // the input table and we have statistics on the table. val columnsUsed = new ColumnPruner(this, table).columnsUsed - val cacheMode = CacheType.fromString( - tableDesc.getProperties().get("shark.cache").asInstanceOf[String]) if (!table.isPartitioned && cacheMode == CacheType.TACHYON) { SharkEnv.tachyonUtil.pushDownColumnPruning(rdd, columnsUsed) } diff --git a/src/test/java/shark/JavaAPISuite.java b/src/test/java/shark/JavaAPISuite.java index 0f05d0fc..49b0d2e8 100644 --- a/src/test/java/shark/JavaAPISuite.java +++ b/src/test/java/shark/JavaAPISuite.java @@ -52,12 +52,6 @@ public static void oneTimeSetUp() { // instantiate a JavaSharkContext. sc = SharkRunner.initWithJava(); - sc.sql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + - METASTORE_PATH + ";create=true"); - sc.sql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH); - - sc.sql("set shark.test.data.path=" + TestUtils$.MODULE$.dataFilePath()); - // test sc.sql("drop table if exists test_java"); sc.sql("CREATE TABLE test_java (key INT, val STRING)"); diff --git a/src/test/scala/shark/SQLSuite.scala b/src/test/scala/shark/SQLSuite.scala index a187d1c9..746e3c18 100644 --- a/src/test/scala/shark/SQLSuite.scala +++ b/src/test/scala/shark/SQLSuite.scala @@ -1038,6 +1038,10 @@ class SQLSuite extends FunSuite { val cachedCount = cachedTableCounts(i) assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) + // Check that we're able to materialize a row - i.e., make sure that table scan operator + // doesn't try to use a ColumnarSerDe when scanning contents on disk (for our test tables, + // LazySimpleSerDes should be used). + sc.sql("select * from %s limit 1".format(tableName)) } // Finally, reload all tables. SharkRunner.loadTables()