From f7ce1b3b48f0354434456241188c6a5d954852e2 Mon Sep 17 00:00:00 2001 From: Neville Li Date: Mon, 7 Jul 2014 15:06:14 -0700 Subject: [PATCH 001/338] [SPARK-1977][MLLIB] register mutable BitSet in MovieLenseALS Author: Neville Li Closes #1319 from nevillelyh/gh/SPARK-1977 and squashes the following commits: 1f0a355 [Neville Li] [SPARK-1977][MLLIB] register mutable BitSet in MovieLenseALS --- .../scala/org/apache/spark/examples/mllib/MovieLensALS.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 28e201d279f41..98aaedb9d7dc9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -17,6 +17,8 @@ package org.apache.spark.examples.mllib +import scala.collection.mutable + import com.esotericsoftware.kryo.Kryo import org.apache.log4j.{Level, Logger} import scopt.OptionParser @@ -41,6 +43,7 @@ object MovieLensALS { class ALSRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo) { kryo.register(classOf[Rating]) + kryo.register(classOf[mutable.BitSet]) } } From c0b4cf097de50eb2c4b0f0e67da53ee92efc1f77 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 7 Jul 2014 17:01:44 -0700 Subject: [PATCH 002/338] [SPARK-2339][SQL] SQL parser in sql-core is case sensitive, but a table alias is converted to lower case when we create Subquery Reported by http://apache-spark-user-list.1001560.n3.nabble.com/Spark-SQL-Join-throws-exception-td8599.html After we get the table from the catalog, because the table has an alias, we will temporarily insert a Subquery. Then, we convert the table alias to lower case no matter if the parser is case sensitive or not. To see the issue ... ``` val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD case class Person(name: String, age: Int) val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) people.registerAsTable("people") sqlContext.sql("select PEOPLE.name from people PEOPLE") ``` The plan is ... ``` == Query Plan == Project ['PEOPLE.name] ExistingRdd [name#0,age#1], MapPartitionsRDD[4] at mapPartitions at basicOperators.scala:176 ``` You can find that `PEOPLE.name` is not resolved. This PR introduces three changes. 1. If a table has an alias, the catalog will not lowercase the alias. If a lowercase alias is needed, the analyzer will do the work. 2. A catalog has a new val caseSensitive that indicates if this catalog is case sensitive or not. For example, a SimpleCatalog is case sensitive, but 3. Corresponding unit tests. With this PR, case sensitivity of database names and table names is handled by the catalog. Case sensitivity of other identifiers are handled by the analyzer. JIRA: https://issues.apache.org/jira/browse/SPARK-2339 Author: Yin Huai Closes #1317 from yhuai/SPARK-2339 and squashes the following commits: 12d8006 [Yin Huai] Handling case sensitivity correctly. This patch introduces three changes. 1. If a table has an alias, the catalog will not lowercase the alias. If a lowercase alias is needed, the analyzer will do the work. 2. A catalog has a new val caseSensitive that indicates if this catalog is case sensitive or not. For example, a SimpleCatalog is case sensitive, but 3. Corresponding unit tests. With this patch, case sensitivity of database names and table names is handled by the catalog. Case sensitivity of other identifiers is handled by the analyzer. --- .../spark/sql/catalyst/analysis/Catalog.scala | 55 +++++++++++---- .../sql/catalyst/analysis/AnalysisSuite.scala | 69 ++++++++++++++++--- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 23 ++++--- ...e table-0-5d14d21a239daa42b086cc895215009a | 14 ++++ .../sql/hive/execution/HiveQuerySuite.scala | 16 +++++ 6 files changed, 149 insertions(+), 30 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index f30b5d816703a..0d05d9808b407 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -25,6 +25,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} * An interface for looking up relations by name. Used by an [[Analyzer]]. */ trait Catalog { + + def caseSensitive: Boolean + def lookupRelation( databaseName: Option[String], tableName: String, @@ -35,22 +38,44 @@ trait Catalog { def unregisterTable(databaseName: Option[String], tableName: String): Unit def unregisterAllTables(): Unit + + protected def processDatabaseAndTableName( + databaseName: Option[String], + tableName: String): (Option[String], String) = { + if (!caseSensitive) { + (databaseName.map(_.toLowerCase), tableName.toLowerCase) + } else { + (databaseName, tableName) + } + } + + protected def processDatabaseAndTableName( + databaseName: String, + tableName: String): (String, String) = { + if (!caseSensitive) { + (databaseName.toLowerCase, tableName.toLowerCase) + } else { + (databaseName, tableName) + } + } } -class SimpleCatalog extends Catalog { +class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { val tables = new mutable.HashMap[String, LogicalPlan]() override def registerTable( databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = { - tables += ((tableName, plan)) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + tables += ((tblName, plan)) } override def unregisterTable( databaseName: Option[String], tableName: String) = { - tables -= tableName + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + tables -= tblName } override def unregisterAllTables() = { @@ -61,12 +86,13 @@ class SimpleCatalog extends Catalog { databaseName: Option[String], tableName: String, alias: Option[String] = None): LogicalPlan = { - val table = tables.get(tableName).getOrElse(sys.error(s"Table Not Found: $tableName")) - val tableWithQualifiers = Subquery(tableName, table) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + val table = tables.get(tblName).getOrElse(sys.error(s"Table Not Found: $tableName")) + val tableWithQualifiers = Subquery(tblName, table) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are // properly qualified with this alias. - alias.map(a => Subquery(a.toLowerCase, tableWithQualifiers)).getOrElse(tableWithQualifiers) + alias.map(a => Subquery(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) } } @@ -85,26 +111,28 @@ trait OverrideCatalog extends Catalog { databaseName: Option[String], tableName: String, alias: Option[String] = None): LogicalPlan = { - - val overriddenTable = overrides.get((databaseName, tableName)) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + val overriddenTable = overrides.get((dbName, tblName)) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are // properly qualified with this alias. val withAlias = - overriddenTable.map(r => alias.map(a => Subquery(a.toLowerCase, r)).getOrElse(r)) + overriddenTable.map(r => alias.map(a => Subquery(a, r)).getOrElse(r)) - withAlias.getOrElse(super.lookupRelation(databaseName, tableName, alias)) + withAlias.getOrElse(super.lookupRelation(dbName, tblName, alias)) } override def registerTable( databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = { - overrides.put((databaseName, tableName), plan) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + overrides.put((dbName, tblName), plan) } override def unregisterTable(databaseName: Option[String], tableName: String): Unit = { - overrides.remove((databaseName, tableName)) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + overrides.remove((dbName, tblName)) } override def unregisterAllTables(): Unit = { @@ -117,6 +145,9 @@ trait OverrideCatalog extends Catalog { * relations are already filled in and the analyser needs only to resolve attribute references. */ object EmptyCatalog extends Catalog { + + val caseSensitive: Boolean = true + def lookupRelation( databaseName: Option[String], tableName: String, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index f14df8137683b..0a4fde3de7752 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,28 +17,81 @@ package org.apache.spark.sql.catalyst.analysis -import org.scalatest.FunSuite +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.types.IntegerType -/* Implicit conversions */ -import org.apache.spark.sql.catalyst.dsl.expressions._ +class AnalysisSuite extends FunSuite with BeforeAndAfter { + val caseSensitiveCatalog = new SimpleCatalog(true) + val caseInsensitiveCatalog = new SimpleCatalog(false) + val caseSensitiveAnalyze = + new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true) + val caseInsensitiveAnalyze = + new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) -class AnalysisSuite extends FunSuite { - val analyze = SimpleAnalyzer + val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) - val testRelation = LocalRelation('a.int) + before { + caseSensitiveCatalog.registerTable(None, "TaBlE", testRelation) + caseInsensitiveCatalog.registerTable(None, "TaBlE", testRelation) + } test("analyze project") { assert( - analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + Project(testRelation.output, testRelation)) + + assert( + caseSensitiveAnalyze( + Project(Seq(UnresolvedAttribute("TbL.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) === + Project(testRelation.output, testRelation)) + + val e = intercept[TreeNodeException[_]] { + caseSensitiveAnalyze( + Project(Seq(UnresolvedAttribute("tBl.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) + } + assert(e.getMessage().toLowerCase.contains("unresolved")) + + assert( + caseInsensitiveAnalyze( + Project(Seq(UnresolvedAttribute("TbL.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) === Project(testRelation.output, testRelation)) + + assert( + caseInsensitiveAnalyze( + Project(Seq(UnresolvedAttribute("tBl.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) === + Project(testRelation.output, testRelation)) + } + + test("resolve relations") { + val e = intercept[RuntimeException] { + caseSensitiveAnalyze(UnresolvedRelation(None, "tAbLe", None)) + } + assert(e.getMessage === "Table Not Found: tAbLe") + + assert( + caseSensitiveAnalyze(UnresolvedRelation(None, "TaBlE", None)) === + testRelation) + + assert( + caseInsensitiveAnalyze(UnresolvedRelation(None, "tAbLe", None)) === + testRelation) + + assert( + caseInsensitiveAnalyze(UnresolvedRelation(None, "TaBlE", None)) === + testRelation) } test("throw errors for unresolved attributes during analysis") { val e = intercept[TreeNodeException[_]] { - analyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) + caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) } assert(e.getMessage().toLowerCase.contains("unresolved")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7edb548678c33..4abd89955bd27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -57,7 +57,7 @@ class SQLContext(@transient val sparkContext: SparkContext) self => @transient - protected[sql] lazy val catalog: Catalog = new SimpleCatalog + protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) @transient protected[sql] lazy val analyzer: Analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 7c24b5cabf614..f83068860701f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -45,12 +45,15 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val client = Hive.get(hive.hiveconf) + val caseSensitive: Boolean = false + def lookupRelation( db: Option[String], tableName: String, alias: Option[String]): LogicalPlan = { - val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase) - val table = client.getTable(databaseName, tableName) + val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) + val table = client.getTable(databaseName, tblName) val partitions: Seq[Partition] = if (table.isPartitioned) { client.getAllPartitionsForPruner(table).toSeq @@ -60,8 +63,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Since HiveQL is case insensitive for table names we make them all lowercase. MetastoreRelation( - databaseName.toLowerCase, - tableName.toLowerCase, + databaseName, + tblName, alias)(table.getTTable, partitions.map(part => part.getTPartition)) } @@ -70,7 +73,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with tableName: String, schema: Seq[Attribute], allowExisting: Boolean = false): Unit = { - val table = new Table(databaseName, tableName) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + val table = new Table(dbName, tblName) val hiveSchema = schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) table.setFields(hiveSchema) @@ -85,7 +89,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with sd.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") sd.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") val serDeInfo = new SerDeInfo() - serDeInfo.setName(tableName) + serDeInfo.setName(tblName) serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") serDeInfo.setParameters(Map[String, String]()) sd.setSerdeInfo(serDeInfo) @@ -104,13 +108,14 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(db, tableName, child) => - val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase) + val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - createTable(databaseName, tableName, child.output) + createTable(databaseName, tblName, child.output) InsertIntoTable( EliminateAnalysisOperators( - lookupRelation(Some(databaseName), tableName, None)), + lookupRelation(Some(databaseName), tblName, None)), Map.empty, child, overwrite = false) diff --git a/sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a b/sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a new file mode 100644 index 0000000000000..4d7127c0faabc --- /dev/null +++ b/sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a @@ -0,0 +1,14 @@ +0 val_0 +4 val_4 +12 val_12 +8 val_8 +0 val_0 +0 val_0 +10 val_10 +5 val_5 +11 val_11 +5 val_5 +2 val_2 +12 val_12 +5 val_5 +9 val_9 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 9f1cd703103ec..a623d29b53973 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -210,6 +210,22 @@ class HiveQuerySuite extends HiveComparisonTest { } } + createQueryTest("case sensitivity: Hive table", + "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") + + test("case sensitivity: registered table") { + val testData: SchemaRDD = + TestHive.sparkContext.parallelize( + TestData(1, "str1") :: + TestData(2, "str2") :: Nil) + testData.registerAsTable("REGisteredTABle") + + assertResult(Array(Array(2, "str2"))) { + hql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + + "WHERE TableAliaS.a > 1").collect() + } + } + def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } explanation.size > 1 && explanation.head.startsWith("Physical execution plan") From 4deeed17c4847f212a4fa1a8685cfe8a12179263 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 7 Jul 2014 17:04:02 -0700 Subject: [PATCH 003/338] [SPARK-2386] [SQL] RowWriteSupport should use the exact types to cast. When execute `saveAsParquetFile` with non-primitive type, `RowWriteSupport` uses wrong type `Int` for `ByteType` and `ShortType`. Author: Takuya UESHIN Closes #1315 from ueshin/issues/SPARK-2386 and squashes the following commits: 20d89ec [Takuya UESHIN] Use None instead of null. bd88741 [Takuya UESHIN] Add a test. 323d1d2 [Takuya UESHIN] Modify RowWriteSupport to use the exact types to cast. --- .../sql/parquet/ParquetTableSupport.scala | 4 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 40 ++++++++++++++++++- 2 files changed, 41 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index bfcbdeb34a92f..9cd5dc5bbd393 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -192,9 +192,9 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { ) ) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) - case ShortType => writer.addInteger(value.asInstanceOf[Int]) + case ShortType => writer.addInteger(value.asInstanceOf[Short]) case LongType => writer.addLong(value.asInstanceOf[Long]) - case ByteType => writer.addInteger(value.asInstanceOf[Int]) + case ByteType => writer.addInteger(value.asInstanceOf[Byte]) case DoubleType => writer.addDouble(value.asInstanceOf[Double]) case FloatType => writer.addFloat(value.asInstanceOf[Float]) case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 2ca0c1cdcbeca..dbf315947ff47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -67,6 +67,19 @@ case class AllDataTypes( byteField: Byte, booleanField: Boolean) +case class AllDataTypesWithNonPrimitiveType( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean, + array: Seq[Int], + map: Map[Int, String], + nested: Nested) + class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. @@ -119,6 +132,31 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } } + test("Read/Write All Types with non-primitive type") { + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + val range = (0 to 255) + TestSQLContext.sparkContext.parallelize(range) + .map(x => AllDataTypesWithNonPrimitiveType( + s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, + Seq(x), Map(x -> s"$x"), Nested(x, s"$x"))) + .saveAsParquetFile(tempDir) + val result = parquetFile(tempDir).collect() + range.foreach { + i => + assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") + assert(result(i).getInt(1) === i) + assert(result(i).getLong(2) === i.toLong) + assert(result(i).getFloat(3) === i.toFloat) + assert(result(i).getDouble(4) === i.toDouble) + assert(result(i).getShort(5) === i.toShort) + assert(result(i).getByte(6) === i.toByte) + assert(result(i).getBoolean(7) === (i % 2 == 0)) + assert(result(i)(8) === Seq(i)) + assert(result(i)(9) === Map(i -> s"$i")) + assert(result(i)(10) === new GenericRow(Array[Any](i, s"$i"))) + } + } + test("self-join parquet files") { val x = ParquetTestData.testData.as('x) val y = ParquetTestData.testData.as('y) @@ -298,7 +336,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("save and load case class RDD with Nones as parquet") { - val data = OptionalReflectData(null, null, null, null, null) + val data = OptionalReflectData(None, None, None, None, None) val rdd = sparkContext.parallelize(data :: Nil) val file = getTempFilePath("parquet") From f0496ee10847db921a028a34f70385f9b740b3f3 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 7 Jul 2014 17:05:59 -0700 Subject: [PATCH 004/338] [SPARK-2375][SQL] JSON schema inference may not resolve type conflicts correctly for a field inside an array of structs For example, for ``` {"array": [{"field":214748364700}, {"field":1}]} ``` the type of field is resolved as IntType. While, for ``` {"array": [{"field":1}, {"field":214748364700}]} ``` the type of field is resolved as LongType. JIRA: https://issues.apache.org/jira/browse/SPARK-2375 Author: Yin Huai Closes #1308 from yhuai/SPARK-2375 and squashes the following commits: 3e2e312 [Yin Huai] Update unit test. 1b2ff9f [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2375 10794eb [Yin Huai] Correctly resolve the type of a field inside an array of structs. --- .../main/scala/org/apache/spark/sql/json/JsonRDD.scala | 9 +++++---- .../test/scala/org/apache/spark/sql/json/JsonSuite.scala | 8 +++++--- .../scala/org/apache/spark/sql/json/TestJsonData.scala | 3 ++- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index edf86775579d8..f6cbca96483e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -198,11 +198,12 @@ private[sql] object JsonRDD extends Logging { * in this JSON object can appear in other JSON objects. */ private def allKeysWithValueTypes(m: Map[String, Any]): Set[(String, DataType)] = { - m.map{ + val keyValuePairs = m.map { // Quote the key with backticks to handle cases which have dots // in the field name. - case (key, dataType) => (s"`$key`", dataType) - }.flatMap { + case (key, value) => (s"`$key`", value) + }.toSet + keyValuePairs.flatMap { case (key: String, struct: Map[String, Any]) => { // The value associted with the key is an JSON object. allKeysWithValueTypes(struct).map { @@ -224,7 +225,7 @@ private[sql] object JsonRDD extends Logging { } } case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil - }.toSet + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 10bd9f08f0238..e765cfc83a397 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -451,7 +451,9 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(arrayElementTypeConflict) val expectedSchema = - AttributeReference("array", ArrayType(StringType), true)() :: Nil + AttributeReference("array1", ArrayType(StringType), true)() :: + AttributeReference("array2", ArrayType(StructType( + StructField("field", LongType, true) :: Nil)), true)() :: Nil comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) @@ -460,12 +462,12 @@ class JsonSuite extends QueryTest { checkAnswer( sql("select * from jsonTable"), Seq(Seq("1", "1.1", "true", null, "[]", "{}", "[2,3,4]", - """{"field":str}""")) :: Nil + """{"field":str}"""), Seq(Seq(214748364700L), Seq(1))) :: Nil ) // Treat an element as a number. checkAnswer( - sql("select array[0] + 1 from jsonTable"), + sql("select array1[0] + 1 from jsonTable"), 2 ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index 065e04046e8a6..d0180f3754f22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -72,7 +72,8 @@ object TestJsonData { val arrayElementTypeConflict = TestSQLContext.sparkContext.parallelize( - """{"array": [1, 1.1, true, null, [], {}, [2,3,4], {"field":"str"}]}""" :: Nil) + """{"array1": [1, 1.1, true, null, [], {}, [2,3,4], {"field":"str"}], + "array2": [{"field":214748364700}, {"field":1}]}""" :: Nil) val missingFields = TestSQLContext.sparkContext.parallelize( From 4352a2fdaa64efee7158eabef65703460ff284ec Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 7 Jul 2014 18:37:38 -0700 Subject: [PATCH 005/338] [SPARK-2376][SQL] Selecting list values inside nested JSON objects raises java.lang.IllegalArgumentException JIRA: https://issues.apache.org/jira/browse/SPARK-2376 Author: Yin Huai Closes #1320 from yhuai/SPARK-2376 and squashes the following commits: 0107417 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2376 480803d [Yin Huai] Correctly handling JSON arrays in PySpark. --- python/pyspark/sql.py | 24 +++++----- .../org/apache/spark/sql/SchemaRDD.scala | 45 ++++++++++++------- 2 files changed, 44 insertions(+), 25 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 5051c82da32a7..ffe177576f363 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -152,10 +152,12 @@ def jsonFile(self, path): >>> ofn.close() >>> srdd = sqlCtx.jsonFile(jsonFile) >>> sqlCtx.registerRDDAsTable(srdd, "table1") - >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1") - >>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}}, - ... {"f1": 2, "f2": "row2", "f3":{"field4":22}}, - ... {"f1": 3, "f2": "row3", "f3":{"field4":33}}] + >>> srdd2 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") + >>> srdd2.collect() == [ + ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, + ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, + ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] True """ jschema_rdd = self._ssql_ctx.jsonFile(path) @@ -167,10 +169,12 @@ def jsonRDD(self, rdd): >>> srdd = sqlCtx.jsonRDD(json) >>> sqlCtx.registerRDDAsTable(srdd, "table1") - >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1") - >>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}}, - ... {"f1": 2, "f2": "row2", "f3":{"field4":22}}, - ... {"f1": 3, "f2": "row3", "f3":{"field4":33}}] + >>> srdd2 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") + >>> srdd2.collect() == [ + ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, + ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, + ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] True """ def func(split, iterator): @@ -492,8 +496,8 @@ def _test(): globs['rdd'] = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) jsonStrings = ['{"field1": 1, "field2": "row1", "field3":{"field4":11}}', - '{"field1" : 2, "field2": "row2", "field3":{"field4":22}}', - '{"field1" : 3, "field2": "row3", "field3":{"field4":33}}'] + '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]}, "field6":[{"field7": "row2"}]}', + '{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}'] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) globs['nestedRdd1'] = sc.parallelize([ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 8f9f54f610e9d..8bcfc7c064c2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,6 +17,11 @@ package org.apache.spark.sql +import java.util.{Map => JMap, List => JList, Set => JSet} + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + import net.razorvine.pickle.Pickler import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext} @@ -27,10 +32,9 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.types.{DataType, StructType, BooleanType} +import org.apache.spark.sql.catalyst.types.{ArrayType, BooleanType, StructType} import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD -import java.util.{Map => JMap} /** * :: AlphaComponent :: @@ -359,6 +363,28 @@ class SchemaRDD( case (obj, (name, dataType)) => dataType match { case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) + case array @ ArrayType(struct: StructType) => + val arrayValues = obj match { + case seq: Seq[Any] => + seq.map(element => rowToMap(element.asInstanceOf[Row], struct)).asJava + case list: JList[Any] => + list.map(element => rowToMap(element.asInstanceOf[Row], struct)) + case set: JSet[Any] => + set.map(element => rowToMap(element.asInstanceOf[Row], struct)) + case array if array != null && array.getClass.isArray => + array.asInstanceOf[Array[Any]].map { + element => rowToMap(element.asInstanceOf[Row], struct) + } + case other => other + } + map.put(name, arrayValues) + case array: ArrayType => { + val arrayValues = obj match { + case seq: Seq[Any] => seq.asJava + case other => other + } + map.put(name, arrayValues) + } case other => map.put(name, obj) } } @@ -366,22 +392,11 @@ class SchemaRDD( map } - // TODO: Actually, the schema of a row should be represented by a StructType instead of - // a Seq[Attribute]. Once we have finished that change, we can just use rowToMap to - // construct the Map for python. - val fields: Seq[(String, DataType)] = this.queryExecution.analyzed.output.map( - field => (field.name, field.dataType)) + val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => - val map: JMap[String, Any] = new java.util.HashMap - row.zip(fields).foreach { case (obj, (name, dataType)) => - dataType match { - case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) - case other => map.put(name, obj) - } - } - map + rowToMap(row, rowSchema) }.grouped(10).map(batched => pickle.dumps(batched.toArray)) } } From 50561f4396be7d641feb2a7a54a374d294628231 Mon Sep 17 00:00:00 2001 From: Yanjie Gao Date: Mon, 7 Jul 2014 19:40:04 -0700 Subject: [PATCH 006/338] [SPARK-2235][SQL]Spark SQL basicOperator add Intersect operator Hi all, I want to submit a basic operator Intersect For example , in sql case select * from table1 intersect select * from table2 So ,i want use this operator support this function in Spark SQL This operator will return the the intersection of SparkPlan child table RDD . JIRA:https://issues.apache.org/jira/browse/SPARK-2235 Author: Yanjie Gao Author: YanjieGao <396154235@qq.com> Closes #1150 from YanjieGao/patch-5 and squashes the following commits: 4629afe [YanjieGao] reformat the code bdc2ac0 [YanjieGao] reformat the code as Michael's suggestion 3b29ad6 [YanjieGao] Merge remote branch 'upstream/master' into patch-5 1cfbfe6 [YanjieGao] refomat some files ea78f33 [YanjieGao] resolve conflict and add annotation on basicOperator and remove HiveQl 0c7cca5 [YanjieGao] modify format problem a802ca8 [YanjieGao] Merge remote branch 'upstream/master' into patch-5 5e374c7 [YanjieGao] resolve conflict in SparkStrategies and basicOperator f7961f6 [Yanjie Gao] update the line less than bdc4a05 [Yanjie Gao] Update basicOperators.scala 0b49837 [Yanjie Gao] delete the annotation f1288b4 [Yanjie Gao] delete annotation e2b64be [Yanjie Gao] Update basicOperators.scala 4dd453e [Yanjie Gao] Update SQLQuerySuite.scala 790765d [Yanjie Gao] Update SparkStrategies.scala ac73e60 [Yanjie Gao] Update basicOperators.scala d4ac5e5 [Yanjie Gao] Update HiveQl.scala 61e88e7 [Yanjie Gao] Update SqlParser.scala 469f099 [Yanjie Gao] Update basicOperators.scala e5bff61 [Yanjie Gao] Spark SQL basicOperator add Intersect operator --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 2 ++ .../sql/catalyst/plans/logical/basicOperators.scala | 5 +++++ .../spark/sql/execution/SparkStrategies.scala | 2 ++ .../apache/spark/sql/execution/basicOperators.scala | 13 +++++++++++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +++++++++++ 5 files changed, 33 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index ecb11129557bd..e5653c5b14ac1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -118,6 +118,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val UNCACHE = Keyword("UNCACHE") protected val UNION = Keyword("UNION") protected val WHERE = Keyword("WHERE") + protected val INTERSECT = Keyword("INTERSECT") protected val EXCEPT = Keyword("EXCEPT") @@ -140,6 +141,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected lazy val query: Parser[LogicalPlan] = ( select * ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } | + INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} | UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 0728fa73fb108..1537de259c5b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -210,3 +210,8 @@ case class Distinct(child: LogicalPlan) extends UnaryNode { case object NoRelation extends LeafNode { override def output = Nil } + +case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + override def output = left.output + override def references = Set.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 9e036e127bd56..7080074a69c07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -275,6 +275,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil case logical.Except(left,right) => execution.Except(planLater(left),planLater(right)) :: Nil + case logical.Intersect(left, right) => + execution.Intersect(planLater(left), planLater(right)) :: Nil case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.NoRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 4b59e0b4e58e4..e8816f0b3cd9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -220,3 +220,16 @@ case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode { } } +/** + * :: DeveloperApi :: + * Returns the rows in left that also appear in right using the built in spark + * intersection function. + */ +@DeveloperApi +case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode { + override def output = children.head.output + + override def execute() = { + left.execute().map(_.copy()).intersection(right.execute().map(_.copy())) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 5c6701e203d17..fa1f32f8a49a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -385,6 +385,17 @@ class SQLQuerySuite extends QueryTest { sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData "), Nil) } + test("INTERSECT") { + checkAnswer( + sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM lowerCaseData"), + (1, "a") :: + (2, "b") :: + (3, "c") :: + (4, "d") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM upperCaseData"), Nil) + } + test("SET commands semantics using sql()") { TestSQLContext.settings.synchronized { clear() From 0128905eea9f8c597ca238b14c18908995511e76 Mon Sep 17 00:00:00 2001 From: Rishi Verma Date: Tue, 8 Jul 2014 00:29:23 -0700 Subject: [PATCH 007/338] Updated programming-guide.md Made sure that readers know the random number generator seed argument, within the 'takeSample' method, is optional. Author: Rishi Verma Closes #1324 from riverma/patch-1 and squashes the following commits: 4699676 [Rishi Verma] Updated programming-guide.md --- docs/programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 06e4c4ce527e1..b09d6347cd1b2 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -975,8 +975,8 @@ for details. Return an array with the first n elements of the dataset. Note that this is currently not executed in parallel. Instead, the driver program computes all the elements. - takeSample(withReplacement, num, seed) - Return an array with a random sample of num elements of the dataset, with or without replacement, using the given random number generator seed. + takeSample(withReplacement, num, [seed]) + Return an array with a random sample of num elements of the dataset, with or without replacement, optionally pre-specifying a random number generator seed. takeOrdered(n, [ordering]) From 3cd5029be709307415f911236472a685e406e763 Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 8 Jul 2014 00:31:42 -0700 Subject: [PATCH 008/338] =?UTF-8?q?Resolve=20sbt=20warnings=20during=20bui?= =?UTF-8?q?ld=20=E2=85=A1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Author: witgo Closes #1153 from witgo/expectResult and squashes the following commits: 97541d8 [witgo] merge master ead26e7 [witgo] Resolve sbt warnings during build --- .../org/apache/spark/AccumulatorSuite.scala | 6 +- .../apache/spark/util/NextIteratorSuite.scala | 38 +++++----- .../spark/util/SizeEstimatorSuite.scala | 70 +++++++++---------- .../spark/sql/columnar/ColumnStatsSuite.scala | 6 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 8 +-- .../columnar/NullableColumnBuilderSuite.scala | 14 ++-- .../compression/BooleanBitSetSuite.scala | 10 +-- .../compression/DictionaryEncodingSuite.scala | 10 +-- .../compression/IntegralDeltaSuite.scala | 16 ++--- .../compression/RunLengthEncodingSuite.scala | 10 +-- 10 files changed, 94 insertions(+), 94 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 3aab88e9e9196..52d1d5277658e 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -61,7 +61,7 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { val acc : Accumulator[Int] = sc.accumulator(0) val d = sc.parallelize(1 to 20) - evaluating {d.foreach{x => acc.value = x}} should produce [Exception] + an [Exception] should be thrownBy {d.foreach{x => acc.value = x}} } test ("add value to collection accumulators") { @@ -87,11 +87,11 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { sc = new SparkContext("local[" + nThreads + "]", "test") val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) val d = sc.parallelize(1 to maxI) - evaluating { + an [SparkException] should be thrownBy { d.foreach { x => acc.value += x } - } should produce [SparkException] + } resetSparkContext() } } diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index cf438a3d72a06..72e81f3f1a884 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -27,45 +27,45 @@ import org.scalatest.Matchers class NextIteratorSuite extends FunSuite with Matchers { test("one iteration") { val i = new StubIterator(Buffer(1)) - i.hasNext should be === true - i.next should be === 1 - i.hasNext should be === false + i.hasNext should be (true) + i.next should be (1) + i.hasNext should be (false) intercept[NoSuchElementException] { i.next() } } test("two iterations") { val i = new StubIterator(Buffer(1, 2)) - i.hasNext should be === true - i.next should be === 1 - i.hasNext should be === true - i.next should be === 2 - i.hasNext should be === false + i.hasNext should be (true) + i.next should be (1) + i.hasNext should be (true) + i.next should be (2) + i.hasNext should be (false) intercept[NoSuchElementException] { i.next() } } test("empty iteration") { val i = new StubIterator(Buffer()) - i.hasNext should be === false + i.hasNext should be (false) intercept[NoSuchElementException] { i.next() } } test("close is called once for empty iterations") { val i = new StubIterator(Buffer()) - i.hasNext should be === false - i.hasNext should be === false - i.closeCalled should be === 1 + i.hasNext should be (false) + i.hasNext should be (false) + i.closeCalled should be (1) } test("close is called once for non-empty iterations") { val i = new StubIterator(Buffer(1, 2)) - i.next should be === 1 - i.next should be === 2 + i.next should be (1) + i.next should be (2) // close isn't called until we check for the next element - i.closeCalled should be === 0 - i.hasNext should be === false - i.closeCalled should be === 1 - i.hasNext should be === false - i.closeCalled should be === 1 + i.closeCalled should be (0) + i.hasNext should be (false) + i.closeCalled should be (1) + i.hasNext should be (false) + i.closeCalled should be (1) } class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index b583a8bd46ae5..f9d1af88f3a13 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -63,53 +63,53 @@ class SizeEstimatorSuite } test("simple classes") { - expectResult(16)(SizeEstimator.estimate(new DummyClass1)) - expectResult(16)(SizeEstimator.estimate(new DummyClass2)) - expectResult(24)(SizeEstimator.estimate(new DummyClass3)) - expectResult(24)(SizeEstimator.estimate(new DummyClass4(null))) - expectResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) + assertResult(16)(SizeEstimator.estimate(new DummyClass1)) + assertResult(16)(SizeEstimator.estimate(new DummyClass2)) + assertResult(24)(SizeEstimator.estimate(new DummyClass3)) + assertResult(24)(SizeEstimator.estimate(new DummyClass4(null))) + assertResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("strings") { - expectResult(40)(SizeEstimator.estimate(DummyString(""))) - expectResult(48)(SizeEstimator.estimate(DummyString("a"))) - expectResult(48)(SizeEstimator.estimate(DummyString("ab"))) - expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assertResult(40)(SizeEstimator.estimate(DummyString(""))) + assertResult(48)(SizeEstimator.estimate(DummyString("a"))) + assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) + assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) } test("primitive arrays") { - expectResult(32)(SizeEstimator.estimate(new Array[Byte](10))) - expectResult(40)(SizeEstimator.estimate(new Array[Char](10))) - expectResult(40)(SizeEstimator.estimate(new Array[Short](10))) - expectResult(56)(SizeEstimator.estimate(new Array[Int](10))) - expectResult(96)(SizeEstimator.estimate(new Array[Long](10))) - expectResult(56)(SizeEstimator.estimate(new Array[Float](10))) - expectResult(96)(SizeEstimator.estimate(new Array[Double](10))) - expectResult(4016)(SizeEstimator.estimate(new Array[Int](1000))) - expectResult(8016)(SizeEstimator.estimate(new Array[Long](1000))) + assertResult(32)(SizeEstimator.estimate(new Array[Byte](10))) + assertResult(40)(SizeEstimator.estimate(new Array[Char](10))) + assertResult(40)(SizeEstimator.estimate(new Array[Short](10))) + assertResult(56)(SizeEstimator.estimate(new Array[Int](10))) + assertResult(96)(SizeEstimator.estimate(new Array[Long](10))) + assertResult(56)(SizeEstimator.estimate(new Array[Float](10))) + assertResult(96)(SizeEstimator.estimate(new Array[Double](10))) + assertResult(4016)(SizeEstimator.estimate(new Array[Int](1000))) + assertResult(8016)(SizeEstimator.estimate(new Array[Long](1000))) } test("object arrays") { // Arrays containing nulls should just have one pointer per element - expectResult(56)(SizeEstimator.estimate(new Array[String](10))) - expectResult(56)(SizeEstimator.estimate(new Array[AnyRef](10))) + assertResult(56)(SizeEstimator.estimate(new Array[String](10))) + assertResult(56)(SizeEstimator.estimate(new Array[AnyRef](10))) // For object arrays with non-null elements, each object should take one pointer plus // however many bytes that class takes. (Note that Array.fill calls the code in its // second parameter separately for each object, so we get distinct objects.) - expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1))) - expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2))) - expectResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3))) - expectResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2))) + assertResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1))) + assertResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2))) + assertResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3))) + assertResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2))) // Past size 100, our samples 100 elements, but we should still get the right size. - expectResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) + assertResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) // If an array contains the *same* element many times, we should only count it once. val d1 = new DummyClass1 - expectResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object - expectResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object + assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object + assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object // Same thing with huge array containing the same element many times. Note that this won't // return exactly 4032 because it can't tell that *all* the elements will equal the first @@ -127,10 +127,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expectResult(40)(SizeEstimator.estimate(DummyString(""))) - expectResult(48)(SizeEstimator.estimate(DummyString("a"))) - expectResult(48)(SizeEstimator.estimate(DummyString("ab"))) - expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assertResult(40)(SizeEstimator.estimate(DummyString(""))) + assertResult(48)(SizeEstimator.estimate(DummyString("a"))) + assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) + assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) resetOrClear("os.arch", arch) } @@ -142,10 +142,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expectResult(56)(SizeEstimator.estimate(DummyString(""))) - expectResult(64)(SizeEstimator.estimate(DummyString("a"))) - expectResult(64)(SizeEstimator.estimate(DummyString("ab"))) - expectResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assertResult(56)(SizeEstimator.estimate(DummyString(""))) + assertResult(64)(SizeEstimator.estimate(DummyString("a"))) + assertResult(64)(SizeEstimator.estimate(DummyString("ab"))) + assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) resetOrClear("os.arch", arch) resetOrClear("spark.test.useCompressedOops", oops) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 78640b876d4aa..6f0d46d816266 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -39,7 +39,7 @@ class ColumnStatsSuite extends FunSuite { test(s"$columnStatsName: empty") { val columnStats = columnStatsClass.newInstance() - expectResult(columnStats.initialBounds, "Wrong initial bounds") { + assertResult(columnStats.initialBounds, "Wrong initial bounds") { (columnStats.lowerBound, columnStats.upperBound) } } @@ -54,8 +54,8 @@ class ColumnStatsSuite extends FunSuite { val values = rows.map(_.head.asInstanceOf[T#JvmType]) val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] - expectResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) - expectResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) + assertResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) + assertResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 71be41056768f..314b7d317ed75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -35,7 +35,7 @@ class ColumnTypeSuite extends FunSuite with Logging { BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => - expectResult(expectedSize, s"Wrong defaultSize for $columnType") { + assertResult(expectedSize, s"Wrong defaultSize for $columnType") { columnType.defaultSize } } @@ -47,7 +47,7 @@ class ColumnTypeSuite extends FunSuite with Logging { value: JvmType, expected: Int) { - expectResult(expected, s"Wrong actualSize for $columnType") { + assertResult(expected, s"Wrong actualSize for $columnType") { columnType.actualSize(value) } } @@ -127,7 +127,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val length = buffer.getInt() assert(length === serializedObj.length) - expectResult(obj, "Deserialized object didn't equal to the original object") { + assertResult(obj, "Deserialized object didn't equal to the original object") { val bytes = new Array[Byte](length) buffer.get(bytes, 0, length) SparkSqlSerializer.deserialize(bytes) @@ -136,7 +136,7 @@ class ColumnTypeSuite extends FunSuite with Logging { buffer.rewind() buffer.putInt(serializedObj.length).put(serializedObj) - expectResult(obj, "Deserialized object didn't equal to the original object") { + assertResult(obj, "Deserialized object didn't equal to the original object") { buffer.rewind() SparkSqlSerializer.deserialize(GENERIC.extract(buffer)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index d9d1e1bfddb75..d8898527baa39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -48,8 +48,8 @@ class NullableColumnBuilderSuite extends FunSuite { val columnBuilder = TestNullableColumnBuilder(columnType) val buffer = columnBuilder.build() - expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) - expectResult(0, "Wrong null count")(buffer.getInt()) + assertResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + assertResult(0, "Wrong null count")(buffer.getInt()) assert(!buffer.hasRemaining) } @@ -63,8 +63,8 @@ class NullableColumnBuilderSuite extends FunSuite { val buffer = columnBuilder.build() - expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) - expectResult(0, "Wrong null count")(buffer.getInt()) + assertResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + assertResult(0, "Wrong null count")(buffer.getInt()) } test(s"$typeName column builder: null values") { @@ -79,11 +79,11 @@ class NullableColumnBuilderSuite extends FunSuite { val buffer = columnBuilder.build() - expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) - expectResult(4, "Wrong null count")(buffer.getInt()) + assertResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + assertResult(4, "Wrong null count")(buffer.getInt()) // For null positions - (1 to 7 by 2).foreach(expectResult(_, "Wrong null position")(buffer.getInt())) + (1 to 7 by 2).foreach(assertResult(_, "Wrong null position")(buffer.getInt())) // For non-null values (0 until 4).foreach { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index 93259a19b9fe7..5fba00480967c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -48,18 +48,18 @@ class BooleanBitSetSuite extends FunSuite { } // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - expectResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) - expectResult(count, "Wrong element count")(buffer.getInt()) + assertResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(count, "Wrong element count")(buffer.getInt()) var word = 0: Long for (i <- 0 until count) { val bit = i % BITS_PER_LONG word = if (bit == 0) buffer.getLong() else word - expectResult(values(i), s"Wrong value in compressed buffer, index=$i") { + assertResult(values(i), s"Wrong value in compressed buffer, index=$i") { (word & ((1: Long) << bit)) != 0 } } @@ -75,7 +75,7 @@ class BooleanBitSetSuite extends FunSuite { if (values.nonEmpty) { values.foreach { assert(decoder.hasNext) - expectResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value")(decoder.next()) } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index 198dcd8819341..d8ae2a26778c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -71,22 +71,22 @@ class DictionaryEncodingSuite extends FunSuite { // 2 bytes for each `Short` val compressedSize = 4 + dictionarySize + 2 * inputSeq.length // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) val dictionary = buildDictionary(buffer).toMap dictValues.foreach { i => - expectResult(i, "Wrong dictionary entry") { + assertResult(i, "Wrong dictionary entry") { dictionary(values(i)) } } inputSeq.foreach { i => - expectResult(i.toShort, "Wrong column element value")(buffer.getShort()) + assertResult(i.toShort, "Wrong column element value")(buffer.getShort()) } // ------------- @@ -101,7 +101,7 @@ class DictionaryEncodingSuite extends FunSuite { if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - expectResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value")(decoder.next()) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 46af6e001c633..17619dcf974e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -69,21 +69,21 @@ class IntegralDeltaSuite extends FunSuite { }) // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) buffer.position(headerSize) - expectResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) if (input.nonEmpty) { - expectResult(Byte.MinValue, "The first byte should be an escaping mark")(buffer.get()) - expectResult(input.head, "The first value is wrong")(columnType.extract(buffer)) + assertResult(Byte.MinValue, "The first byte should be an escaping mark")(buffer.get()) + assertResult(input.head, "The first value is wrong")(columnType.extract(buffer)) (input.tail, deltas).zipped.foreach { (value, delta) => if (math.abs(delta) <= Byte.MaxValue) { - expectResult(delta, "Wrong delta")(buffer.get()) + assertResult(delta, "Wrong delta")(buffer.get()) } else { - expectResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) - expectResult(value, "Wrong value")(columnType.extract(buffer)) + assertResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) + assertResult(value, "Wrong value")(columnType.extract(buffer)) } } } @@ -99,7 +99,7 @@ class IntegralDeltaSuite extends FunSuite { if (input.nonEmpty) { input.foreach{ assert(decoder.hasNext) - expectResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value")(decoder.next()) } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index d3b73ba19d476..40115beb98899 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -61,15 +61,15 @@ class RunLengthEncodingSuite extends FunSuite { }.sum // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) inputRuns.foreach { case (index, run) => - expectResult(values(index), "Wrong column element value")(columnType.extract(buffer)) - expectResult(run, "Wrong run length")(buffer.getInt()) + assertResult(values(index), "Wrong column element value")(columnType.extract(buffer)) + assertResult(run, "Wrong run length")(buffer.getInt()) } // ------------- @@ -84,7 +84,7 @@ class RunLengthEncodingSuite extends FunSuite { if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - expectResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value")(decoder.next()) } } From 5a4063645dd7bb4cd8bda890785235729804ab09 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 8 Jul 2014 00:41:46 -0700 Subject: [PATCH 009/338] [SPARK-2391][SQL] Custom take() for LIMIT queries. Using Spark's take can result in an entire in-memory partition to be shipped in order to retrieve a single row. Author: Michael Armbrust Closes #1318 from marmbrus/takeLimit and squashes the following commits: 77289a5 [Michael Armbrust] Update scala doc 32f0674 [Michael Armbrust] Custom take implementation for LIMIT queries. --- .../spark/sql/execution/basicOperators.scala | 51 +++++++++++++++++-- 1 file changed, 47 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index e8816f0b3cd9c..97abd636ab5fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import scala.collection.mutable.ArrayBuffer import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.DeveloperApi @@ -83,9 +84,9 @@ case class Union(children: Seq[SparkPlan])(@transient sqlContext: SQLContext) ex * :: DeveloperApi :: * Take the first limit elements. Note that the implementation is different depending on whether * this is a terminal operator or not. If it is terminal and is invoked using executeCollect, - * this operator uses Spark's take method on the Spark driver. If it is not terminal or is - * invoked using execute, we first take the limit on each partition, and then repartition all the - * data to a single partition to compute the global limit. + * this operator uses something similar to Spark's take method on the Spark driver. If it is not + * terminal or is invoked using execute, we first take the limit on each partition, and then + * repartition all the data to a single partition to compute the global limit. */ @DeveloperApi case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext) @@ -97,7 +98,49 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext override def output = child.output - override def executeCollect() = child.execute().map(_.copy()).take(limit) + /** + * A custom implementation modeled after the take function on RDDs but which never runs any job + * locally. This is to avoid shipping an entire partition of data in order to retrieve only a few + * rows. + */ + override def executeCollect(): Array[Row] = { + if (limit == 0) { + return new Array[Row](0) + } + + val childRDD = child.execute().map(_.copy()) + + val buf = new ArrayBuffer[Row] + val totalParts = childRDD.partitions.length + var partsScanned = 0 + while (buf.size < limit && partsScanned < 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 = 1 + if (partsScanned > 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 (buf.size == 0) { + numPartsToTry = totalParts - 1 + } else { + numPartsToTry = (1.5 * limit * partsScanned / buf.size).toInt + } + } + numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions + + val left = limit - buf.size + val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) + val sc = sqlContext.sparkContext + val res = + sc.runJob(childRDD, (it: Iterator[Row]) => it.take(left).toArray, p, allowLocal = false) + + res.foreach(buf ++= _.take(limit - buf.size)) + partsScanned += numPartsToTry + } + + buf.toArray + } override def execute() = { val rdd = child.execute().mapPartitions { iter => From 56e009d4f05d990c60e109838fa70457f97f44aa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 16:49:31 +0900 Subject: [PATCH 010/338] [EC2] Add default history server port to ec2 script Right now I have to open it manually Author: Andrew Or Closes #1296 from andrewor14/hist-serv-port and squashes the following commits: 8895a1f [Andrew Or] Add default history server port to ec2 script --- ec2/spark_ec2.py | 1 + 1 file changed, 1 insertion(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index e22d93bd31bc2..f5c2bfb697c81 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -279,6 +279,7 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize(src_group=slave_group) master_group.authorize('tcp', 22, 22, '0.0.0.0/0') master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') + master_group.authorize('tcp', 18080, 18080, '0.0.0.0/0') master_group.authorize('tcp', 19999, 19999, '0.0.0.0/0') master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') From cc3e0a14daf756ff5c2d4e7916438e175046e5bb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 8 Jul 2014 10:36:18 -0700 Subject: [PATCH 011/338] [SPARK-2395][SQL] Optimize common LIKE patterns. Author: Michael Armbrust Closes #1325 from marmbrus/slowLike and squashes the following commits: 023c3eb [Michael Armbrust] add comment. 8b421c2 [Michael Armbrust] Handle the case where the final % is actually escaped. d34d37e [Michael Armbrust] add periods. 3bbf35f [Michael Armbrust] Roll back changes to SparkBuild 53894b1 [Michael Armbrust] Fix grammar. 4094462 [Michael Armbrust] Fix grammar. 6d3d0a0 [Michael Armbrust] Optimize common LIKE patterns. --- .../expressions/stringOperations.scala | 51 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 23 +++++++++ 2 files changed, 74 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index c074b7bb01e57..347471cebdc7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -156,3 +156,54 @@ case class Lower(child: Expression) extends UnaryExpression with CaseConversionE override def toString() = s"Lower($child)" } + +/** A base class for functions that compare two strings, returning a boolean. */ +abstract class StringComparison extends Expression { + self: Product => + + type EvaluatedType = Any + + def left: Expression + def right: Expression + + override def references = children.flatMap(_.references).toSet + override def children = left :: right :: Nil + + override def nullable: Boolean = true + override def dataType: DataType = BooleanType + + def compare(l: String, r: String): Boolean + + override def eval(input: Row): Any = { + val leftEval = left.eval(input).asInstanceOf[String] + if(leftEval == null) { + null + } else { + val rightEval = right.eval(input).asInstanceOf[String] + if (rightEval == null) null else compare(leftEval, rightEval) + } + } + + override def toString() = s"$nodeName($left, $right)" +} + +/** + * A function that returns true if the string `left` contains the string `right`. + */ +case class Contains(left: Expression, right: Expression) extends StringComparison { + override def compare(l: String, r: String) = l.contains(r) +} + +/** + * A function that returns true if the string `left` starts with the string `right`. + */ +case class StartsWith(left: Expression, right: Expression) extends StringComparison { + def compare(l: String, r: String) = l.startsWith(r) +} + +/** + * A function that returns true if the string `left` ends with the string `right`. + */ +case class EndsWith(left: Expression, right: Expression) extends StringComparison { + def compare(l: String, r: String) = l.endsWith(r) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 48ca31e70fe7c..f0904f59d028f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -34,6 +34,7 @@ object Optimizer extends RuleExecutor[LogicalPlan] { Batch("ConstantFolding", FixedPoint(100), NullPropagation, ConstantFolding, + LikeSimplification, BooleanSimplification, SimplifyFilters, SimplifyCasts, @@ -111,6 +112,28 @@ object ColumnPruning extends Rule[LogicalPlan] { } } +/** + * Simplifies LIKE expressions that do not need full regular expressions to evaluate the condition. + * For example, when the expression is just checking to see if a string starts with a given + * pattern. + */ +object LikeSimplification extends Rule[LogicalPlan] { + // if guards below protect from escapes on trailing %. + // Cases like "something\%" are not optimized, but this does not affect correctness. + val startsWith = "([^_%]+)%".r + val endsWith = "%([^_%]+)".r + val contains = "%([^_%]+)%".r + + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case Like(l, Literal(startsWith(pattern), StringType)) if !pattern.endsWith("\\") => + StartsWith(l, Literal(pattern)) + case Like(l, Literal(endsWith(pattern), StringType)) => + EndsWith(l, Literal(pattern)) + case Like(l, Literal(contains(pattern), StringType)) if !pattern.endsWith("\\") => + Contains(l, Literal(pattern)) + } +} + /** * Replaces [[Expression Expressions]] that can be statically evaluated with * equivalent [[Literal]] values. This rule is more specific with From c8a2313cdf825e0191680a423d17619b5504ff89 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Tue, 8 Jul 2014 10:43:46 -0700 Subject: [PATCH 012/338] [SPARK-2403] Catch all errors during serialization in DAGScheduler https://issues.apache.org/jira/browse/SPARK-2403 Spark hangs for us whenever we forget to register a class with Kryo. This should be a simple fix for that. But let me know if you have a better suggestion. I did not write a new test for this. It would be pretty complicated and I'm not sure it's worthwhile for such a simple change. Let me know if you disagree. Author: Daniel Darabos Closes #1329 from darabos/spark-2403 and squashes the following commits: 3aceaad [Daniel Darabos] Print full stack trace for miscellaneous exceptions during serialization. 52c22ba [Daniel Darabos] Only catch NonFatal exceptions. 361e962 [Daniel Darabos] Catch all errors during serialization in DAGScheduler. --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 81c136d970312..f72bfde572c96 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -26,6 +26,7 @@ import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps import scala.reflect.ClassTag +import scala.util.control.NonFatal import akka.actor._ import akka.actor.OneForOneStrategy @@ -768,6 +769,10 @@ class DAGScheduler( abortStage(stage, "Task not serializable: " + e.toString) runningStages -= stage return + case NonFatal(e) => // Other exceptions, such as IllegalArgumentException from Kryo. + abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") + runningStages -= stage + return } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") From b520b6453ed76926108e0bdd56114d16e1d86850 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Tue, 8 Jul 2014 13:55:42 -0500 Subject: [PATCH 013/338] SPARK-2400 : fix spark.yarn.max.executor.failures explaination According to ```scala private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) ``` default value should be numExecutors * 2, with minimum of 3, and it's same to the config `spark.yarn.max.worker.failures` Author: CrazyJvm Closes #1282 from CrazyJvm/yarn-doc and squashes the following commits: 1a5f25b [CrazyJvm] remove deprecated config c438aec [CrazyJvm] fix style 86effa6 [CrazyJvm] change expression 211f130 [CrazyJvm] fix html tag 2900d23 [CrazyJvm] fix style a4b2e27 [CrazyJvm] fix configuration spark.yarn.max.executor.failures --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 5d8d603aa3e37..0362f5a223319 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -55,7 +55,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.max.executor.failures - 2*numExecutors + numExecutors * 2, with minimum of 3 The maximum number of executor failures before failing the application. From 32516f866a32d51bfaa04685ae77ba216b4202d9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 8 Jul 2014 14:00:47 -0700 Subject: [PATCH 014/338] [SPARK-2409] Make SQLConf thread safe. Author: Reynold Xin Closes #1334 from rxin/sqlConfThreadSafetuy and squashes the following commits: c1e0a5a [Reynold Xin] Fixed the duplicate comment. 7614372 [Reynold Xin] [SPARK-2409] Make SQLConf thread safe. --- .../src/main/scala/org/apache/spark/sql/SQLConf.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 95ed0f28507fc..b6fb46a3acc03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -25,7 +25,9 @@ import scala.collection.JavaConverters._ * SQLConf holds mutable config parameters and hints. These can be set and * queried either by passing SET commands into Spark SQL's DSL * functions (sql(), hql(), etc.), or by programmatically using setters and - * getters of this class. This class is thread-safe. + * getters of this class. + * + * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { @@ -71,11 +73,9 @@ trait SQLConf { Option(settings.get(key)).getOrElse(defaultValue) } - def getAll: Array[(String, String)] = settings.asScala.toArray + def getAll: Array[(String, String)] = settings.synchronized { settings.asScala.toArray } - def getOption(key: String): Option[String] = { - Option(settings.get(key)) - } + def getOption(key: String): Option[String] = Option(settings.get(key)) def contains(key: String): Boolean = settings.containsKey(key) From e6f7bfcfbf6aff7a9f8cd8e0a2166d0bf62b0912 Mon Sep 17 00:00:00 2001 From: Gabriele Nizzoli Date: Tue, 8 Jul 2014 14:23:38 -0700 Subject: [PATCH 015/338] [SPARK-2362] Fix for newFilesOnly logic in file DStream The newFilesOnly logic should be inverted: the logic should be that if the flag newFilesOnly==true then only start reading files older than current time. As the code is now if newFilesOnly==true then it will start to read files that are older than 0L (that is: every file in the directory). Author: Gabriele Nizzoli Closes #1077 from gabrielenizzoli/master and squashes the following commits: 4f1d261 [Gabriele Nizzoli] Fix for newFilesOnly logic in file DStream --- .../org/apache/spark/streaming/dstream/FileInputDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index e878285f6a854..9eecbfaef363f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -45,7 +45,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas // Files with mod time earlier than this is ignored. This is updated every interval // such that in the current interval, files older than any file found in the // previous interval will be ignored. Obviously this time keeps moving forward. - private var ignoreTime = if (newFilesOnly) 0L else System.currentTimeMillis() + private var ignoreTime = if (newFilesOnly) System.currentTimeMillis() else 0L // Latest file mod time seen till any point of time @transient private var path_ : Path = null From bf04a390e40d60aa7fcc551501d25f3f9d38377c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 17:35:31 -0700 Subject: [PATCH 016/338] [SPARK-2392] Executors should not start their own HTTP servers Executors currently start their own unused HTTP file servers. This is because we use the same SparkEnv class for both executors and drivers, and we do not distinguish this case. In the longer term, we should separate out SparkEnv for the driver and SparkEnv for the executors. Author: Andrew Or Closes #1335 from andrewor14/executor-http-server and squashes the following commits: 46ef263 [Andrew Or] Start HTTP server only on the driver --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2b636b085d73a..8f70744d804d9 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -79,7 +79,7 @@ class SparkEnv ( private[spark] def stop() { pythonWorkers.foreach { case(key, worker) => worker.stop() } - httpFileServer.stop() + Option(httpFileServer).foreach(_.stop()) mapOutputTracker.stop() shuffleManager.stop() broadcastManager.stop() @@ -228,9 +228,15 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - val httpFileServer = new HttpFileServer(securityManager) - httpFileServer.initialize() - conf.set("spark.fileserver.uri", httpFileServer.serverUri) + val httpFileServer = + if (isDriver) { + val server = new HttpFileServer(securityManager) + server.initialize() + conf.set("spark.fileserver.uri", server.serverUri) + server + } else { + null + } val metricsSystem = if (isDriver) { MetricsSystem.createMetricsSystem("driver", conf, securityManager) From ac9cdc116e1c5fcb291a4ff168cac002a8058f05 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 8 Jul 2014 17:50:36 -0700 Subject: [PATCH 017/338] [SPARK-2413] Upgrade junit_xml_listener to 0.5.1 which fixes the following issues 1) fix the class name to be fully qualified classpath 2) make sure the the reporting time is in second not in miliseond, which causing JUnit HTML to report incorrect number 3) make sure the duration of the tests are accumulative. Author: DB Tsai Closes #1333 from dbtsai/dbtsai-junit and squashes the following commits: bbeac4b [DB Tsai] Upgrade junit_xml_listener to 0.5.1 which fixes the following issues --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 472819b9fb8ba..d3ac4bf335e87 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -21,6 +21,6 @@ addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") -addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") +addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") From 1114207cc8e4ef94cb97bbd5a2ef3ae4d51f73fa Mon Sep 17 00:00:00 2001 From: johnnywalleye Date: Tue, 8 Jul 2014 19:17:26 -0700 Subject: [PATCH 018/338] [SPARK-2152][MLlib] fix bin offset in DecisionTree node aggregations (also resolves SPARK-2160) Hi, this pull fixes (what I believe to be) a bug in DecisionTree.scala. In the extractLeftRightNodeAggregates function, the first set of rightNodeAgg values for Regression are set in line 792 as follows: rightNodeAgg(featureIndex)(2 * (numBins - 2)) = binData(shift + (2 * numBins - 1))) Then there is a loop that sets the rest of the values, as in line 809: rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) = binData(shift + (2 *(numBins - 2 - splitIndex))) + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex)) But since splitIndex starts at 1, this ends up skipping a set of binData values. The changes here address this issue, for both the Regression and Classification cases. Author: johnnywalleye Closes #1316 from johnnywalleye/master and squashes the following commits: 73809da [johnnywalleye] fix bin offset in DecisionTree node aggregations --- .../org/apache/spark/mllib/tree/DecisionTree.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 3b13e52a7b445..74d5d7ba10960 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -807,10 +807,10 @@ object DecisionTree extends Serializable with Logging { // calculating right node aggregate for a split as a sum of right node aggregate of a // higher split and the right bin aggregate of a bin where the split is a low split rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) = - binData(shift + (2 *(numBins - 2 - splitIndex))) + + binData(shift + (2 *(numBins - 1 - splitIndex))) + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex)) rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (2* (numBins - 2 - splitIndex) + 1)) + + binData(shift + (2* (numBins - 1 - splitIndex) + 1)) + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex) + 1) splitIndex += 1 @@ -855,13 +855,13 @@ object DecisionTree extends Serializable with Logging { // calculating right node aggregate for a split as a sum of right node aggregate of a // higher split and the right bin aggregate of a bin where the split is a low split rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex)) = - binData(shift + (3 * (numBins - 2 - splitIndex))) + + binData(shift + (3 * (numBins - 1 - splitIndex))) + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex)) rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (3 * (numBins - 2 - splitIndex) + 1)) + + binData(shift + (3 * (numBins - 1 - splitIndex) + 1)) + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 1) rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 2) = - binData(shift + (3 * (numBins - 2 - splitIndex) + 2)) + + binData(shift + (3 * (numBins - 1 - splitIndex) + 2)) + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 2) splitIndex += 1 From 339441f545ca96de4a7d6271e32f16803539a8c8 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Jul 2014 22:57:21 -0700 Subject: [PATCH 019/338] [SPARK-2384] Add tooltips to UI. This patch adds tooltips to clarify some points of confusion in the UI. When users mouse over some of the table headers (shuffle read, write, and input size) as well as over the "scheduler delay" metric shown for each stage, a black tool tip (see image below) pops up describing the metric in more detail. After the tooltip mechanism is added by this commit, I imagine others may want to add more tooltips for other things in the UI, but I think this is a good starting point. ![tooltip](https://cloud.githubusercontent.com/assets/1108612/3491905/994e179e-059f-11e4-92f2-c6c12d248d81.jpg) This looks scary-big but much of it is adding the bootstrap tool tip JavaScript. Also I have no idea what to put for the license in tooltip (I left it the same -- the Twitter apache header) or for JQuery (left it as nothing) -- @mateiz what's the right thing here? cc @pwendell @andrewor14 @rxin Author: Kay Ousterhout Closes #1314 from kayousterhout/tooltips and squashes the following commits: 19981b5 [Kay Ousterhout] Exclude non-licensed javascript files from style check d9ab5a9 [Kay Ousterhout] Response to Andrew's review 7752449 [Kay Ousterhout] [SPARK-2384] Add tooltips to UI. --- .rat-excludes | 2 + .../spark/ui/static/bootstrap-tooltip.js | 288 ++++++++++++++++++ .../spark/ui/static/initialize-tooltips.js | 21 ++ .../spark/ui/static/jquery-1.11.1.min.js | 4 + .../org/apache/spark/ui/static/webui.css | 5 + .../scala/org/apache/spark/ui/ToolTips.scala | 34 +++ .../scala/org/apache/spark/ui/UIUtils.scala | 32 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 136 +++++---- .../apache/spark/ui/jobs/ExecutorTable.scala | 10 +- .../org/apache/spark/ui/jobs/StagePage.scala | 37 +-- .../org/apache/spark/ui/jobs/StageTable.scala | 15 +- 11 files changed, 491 insertions(+), 93 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js create mode 100644 core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js create mode 100644 core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js create mode 100644 core/src/main/scala/org/apache/spark/ui/ToolTips.scala diff --git a/.rat-excludes b/.rat-excludes index 796c32a80896c..372bc2587ccc3 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -22,6 +22,8 @@ slaves spark-env.sh spark-env.sh.template log4j-defaults.properties +bootstrap-tooltip.js +jquery-1.11.1.min.js sorttable.js .*txt .*json diff --git a/core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js b/core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js new file mode 100644 index 0000000000000..2934181c1006a --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js @@ -0,0 +1,288 @@ +/* =========================================================== + * bootstrap-tooltip.js v2.2.2 + * http://twitter.github.com/bootstrap/javascript.html#tooltips + * Inspired by the original jQuery.tipsy by Jason Frame + * =========================================================== + * Copyright 2012 Twitter, Inc. + * + * 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. + * ========================================================== */ + + +!function ($) { + + "use strict"; // jshint ;_; + + + /* TOOLTIP PUBLIC CLASS DEFINITION + * =============================== */ + + var Tooltip = function (element, options) { + this.init('tooltip', element, options) + } + + Tooltip.prototype = { + + constructor: Tooltip + + , init: function (type, element, options) { + var eventIn + , eventOut + + this.type = type + this.$element = $(element) + this.options = this.getOptions(options) + this.enabled = true + + if (this.options.trigger == 'click') { + this.$element.on('click.' + this.type, this.options.selector, $.proxy(this.toggle, this)) + } else if (this.options.trigger != 'manual') { + eventIn = this.options.trigger == 'hover' ? 'mouseenter' : 'focus' + eventOut = this.options.trigger == 'hover' ? 'mouseleave' : 'blur' + this.$element.on(eventIn + '.' + this.type, this.options.selector, $.proxy(this.enter, this)) + this.$element.on(eventOut + '.' + this.type, this.options.selector, $.proxy(this.leave, this)) + } + + this.options.selector ? + (this._options = $.extend({}, this.options, { trigger: 'manual', selector: '' })) : + this.fixTitle() + } + + , getOptions: function (options) { + options = $.extend({}, $.fn[this.type].defaults, options, this.$element.data()) + + if (options.delay && typeof options.delay == 'number') { + options.delay = { + show: options.delay + , hide: options.delay + } + } + + return options + } + + , enter: function (e) { + var self = $(e.currentTarget)[this.type](this._options).data(this.type) + + if (!self.options.delay || !self.options.delay.show) return self.show() + + clearTimeout(this.timeout) + self.hoverState = 'in' + this.timeout = setTimeout(function() { + if (self.hoverState == 'in') self.show() + }, self.options.delay.show) + } + + , leave: function (e) { + var self = $(e.currentTarget)[this.type](this._options).data(this.type) + + if (this.timeout) clearTimeout(this.timeout) + if (!self.options.delay || !self.options.delay.hide) return self.hide() + + self.hoverState = 'out' + this.timeout = setTimeout(function() { + if (self.hoverState == 'out') self.hide() + }, self.options.delay.hide) + } + + , show: function () { + var $tip + , inside + , pos + , actualWidth + , actualHeight + , placement + , tp + + if (this.hasContent() && this.enabled) { + $tip = this.tip() + this.setContent() + + if (this.options.animation) { + $tip.addClass('fade') + } + + placement = typeof this.options.placement == 'function' ? + this.options.placement.call(this, $tip[0], this.$element[0]) : + this.options.placement + + inside = /in/.test(placement) + + $tip + .detach() + .css({ top: 0, left: 0, display: 'block' }) + .insertAfter(this.$element) + + pos = this.getPosition(inside) + + actualWidth = $tip[0].offsetWidth + actualHeight = $tip[0].offsetHeight + + switch (inside ? placement.split(' ')[1] : placement) { + case 'bottom': + tp = {top: pos.top + pos.height, left: pos.left + pos.width / 2 - actualWidth / 2} + break + case 'top': + tp = {top: pos.top - actualHeight, left: pos.left + pos.width / 2 - actualWidth / 2} + break + case 'left': + tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left - actualWidth} + break + case 'right': + tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left + pos.width} + break + } + + $tip + .offset(tp) + .addClass(placement) + .addClass('in') + } + } + + , setContent: function () { + var $tip = this.tip() + , title = this.getTitle() + + $tip.find('.tooltip-inner')[this.options.html ? 'html' : 'text'](title) + $tip.removeClass('fade in top bottom left right') + } + + , hide: function () { + var that = this + , $tip = this.tip() + + $tip.removeClass('in') + + function removeWithAnimation() { + var timeout = setTimeout(function () { + $tip.off($.support.transition.end).detach() + }, 500) + + $tip.one($.support.transition.end, function () { + clearTimeout(timeout) + $tip.detach() + }) + } + + $.support.transition && this.$tip.hasClass('fade') ? + removeWithAnimation() : + $tip.detach() + + return this + } + + , fixTitle: function () { + var $e = this.$element + if ($e.attr('title') || typeof($e.attr('data-original-title')) != 'string') { + $e.attr('data-original-title', $e.attr('title') || '').attr('title', '') + } + } + + , hasContent: function () { + return this.getTitle() + } + + , getPosition: function (inside) { + return $.extend({}, (inside ? {top: 0, left: 0} : this.$element.offset()), { + width: this.$element[0].offsetWidth + , height: this.$element[0].offsetHeight + }) + } + + , getTitle: function () { + var title + , $e = this.$element + , o = this.options + + title = $e.attr('data-original-title') + || (typeof o.title == 'function' ? o.title.call($e[0]) : o.title) + + return title + } + + , tip: function () { + return this.$tip = this.$tip || $(this.options.template) + } + + , validate: function () { + if (!this.$element[0].parentNode) { + this.hide() + this.$element = null + this.options = null + } + } + + , enable: function () { + this.enabled = true + } + + , disable: function () { + this.enabled = false + } + + , toggleEnabled: function () { + this.enabled = !this.enabled + } + + , toggle: function (e) { + var self = $(e.currentTarget)[this.type](this._options).data(this.type) + self[self.tip().hasClass('in') ? 'hide' : 'show']() + } + + , destroy: function () { + this.hide().$element.off('.' + this.type).removeData(this.type) + } + + } + + + /* TOOLTIP PLUGIN DEFINITION + * ========================= */ + + var old = $.fn.tooltip + + $.fn.tooltip = function ( option ) { + return this.each(function () { + var $this = $(this) + , data = $this.data('tooltip') + , options = typeof option == 'object' && option + if (!data) $this.data('tooltip', (data = new Tooltip(this, options))) + if (typeof option == 'string') data[option]() + }) + } + + $.fn.tooltip.Constructor = Tooltip + + $.fn.tooltip.defaults = { + animation: true + , placement: 'top' + , selector: false + , template: '
' + , trigger: 'hover' + , title: '' + , delay: 0 + , html: false + } + + + /* TOOLTIP NO CONFLICT + * =================== */ + + $.fn.tooltip.noConflict = function () { + $.fn.tooltip = old + return this + } + +}(window.jQuery); + diff --git a/core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js b/core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js new file mode 100644 index 0000000000000..70f355dfb49cb --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js @@ -0,0 +1,21 @@ +/* + * 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. + */ + +$(document).ready(function(){ + $("[data-toggle=tooltip]").tooltip({container: 'body'}); +}); + diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js new file mode 100644 index 0000000000000..ab28a24729b32 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js @@ -0,0 +1,4 @@ +/*! jQuery v1.11.1 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l="1.11.1",m=function(a,b){return new m.fn.init(a,b)},n=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,o=/^-ms-/,p=/-([\da-z])/gi,q=function(a,b){return b.toUpperCase()};m.fn=m.prototype={jquery:l,constructor:m,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=m.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return m.each(this,a,b)},map:function(a){return this.pushStack(m.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},m.extend=m.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||m.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(m.isPlainObject(c)||(b=m.isArray(c)))?(b?(b=!1,f=a&&m.isArray(a)?a:[]):f=a&&m.isPlainObject(a)?a:{},g[d]=m.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},m.extend({expando:"jQuery"+(l+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===m.type(a)},isArray:Array.isArray||function(a){return"array"===m.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return!m.isArray(a)&&a-parseFloat(a)>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==m.type(a)||a.nodeType||m.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(k.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&m.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(o,"ms-").replace(p,q)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=r(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(n,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(r(Object(a))?m.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=r(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),m.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||m.guid++,e):void 0},now:function(){return+new Date},support:k}),m.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function r(a){var b=a.length,c=m.type(a);return"function"===c||m.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var s=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+-new Date,v=a.document,w=0,x=0,y=gb(),z=gb(),A=gb(),B=function(a,b){return a===b&&(l=!0),0},C="undefined",D=1<<31,E={}.hasOwnProperty,F=[],G=F.pop,H=F.push,I=F.push,J=F.slice,K=F.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},L="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",M="[\\x20\\t\\r\\n\\f]",N="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=N.replace("w","w#"),P="\\["+M+"*("+N+")(?:"+M+"*([*^$|!~]?=)"+M+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+O+"))|)"+M+"*\\]",Q=":("+N+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+P+")*)|.*)\\)|)",R=new RegExp("^"+M+"+|((?:^|[^\\\\])(?:\\\\.)*)"+M+"+$","g"),S=new RegExp("^"+M+"*,"+M+"*"),T=new RegExp("^"+M+"*([>+~]|"+M+")"+M+"*"),U=new RegExp("="+M+"*([^\\]'\"]*?)"+M+"*\\]","g"),V=new RegExp(Q),W=new RegExp("^"+O+"$"),X={ID:new RegExp("^#("+N+")"),CLASS:new RegExp("^\\.("+N+")"),TAG:new RegExp("^("+N.replace("w","w*")+")"),ATTR:new RegExp("^"+P),PSEUDO:new RegExp("^"+Q),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+L+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),db=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{I.apply(F=J.call(v.childNodes),v.childNodes),F[v.childNodes.length].nodeType}catch(eb){I={apply:F.length?function(a,b){H.apply(a,J.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],!a||"string"!=typeof a)return d;if(1!==(k=b.nodeType)&&9!==k)return[];if(p&&!e){if(f=_.exec(a))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return I.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return I.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&(!q||!q.test(a))){if(s=r=u,w=b,x=9===k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+qb(o[l]);w=ab.test(a)&&ob(b.parentNode)||b,x=o.join(",")}if(x)try{return I.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function gb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function hb(a){return a[u]=!0,a}function ib(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function jb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function kb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||D)-(~a.sourceIndex||D);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function lb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function nb(a){return hb(function(b){return b=+b,hb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function ob(a){return a&&typeof a.getElementsByTagName!==C&&a}c=fb.support={},f=fb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fb.setDocument=function(a){var b,e=a?a.ownerDocument||a:v,g=e.defaultView;return e!==n&&9===e.nodeType&&e.documentElement?(n=e,o=e.documentElement,p=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){m()},!1):g.attachEvent&&g.attachEvent("onunload",function(){m()})),c.attributes=ib(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ib(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=$.test(e.getElementsByClassName)&&ib(function(a){return a.innerHTML="
",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=ib(function(a){return o.appendChild(a).id=u,!e.getElementsByName||!e.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==C&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c=typeof a.getAttributeNode!==C&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==C?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return typeof b.getElementsByClassName!==C&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(e.querySelectorAll))&&(ib(function(a){a.innerHTML="",a.querySelectorAll("[msallowclip^='']").length&&q.push("[*^$]="+M+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+M+"*(?:value|"+L+")"),a.querySelectorAll(":checked").length||q.push(":checked")}),ib(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+M+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ib(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",Q)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===v&&t(v,a)?-1:b===e||b.ownerDocument===v&&t(v,b)?1:k?K.call(k,a)-K.call(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],i=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:k?K.call(k,a)-K.call(k,b):0;if(f===g)return kb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]===i[d])d++;return d?kb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},e):n},fb.matches=function(a,b){return fb(a,null,null,b)},fb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fb(b,n,null,[a]).length>0},fb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&E.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fb.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fb.selectors={cacheLength:50,createPseudo:hb,match:X,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fb.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+M+")"+a+"("+M+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==C&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fb.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(s&&((l[u]||(l[u]={}))[a]=[w,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?hb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=K.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:hb(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));return d[u]?hb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:hb(function(a){return function(b){return fb(a,b).length>0}}),contains:hb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:hb(function(a){return W.test(a||"")||fb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Z.test(a.nodeName)},input:function(a){return Y.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:nb(function(){return[0]}),last:nb(function(a,b){return[b-1]}),eq:nb(function(a,b,c){return[0>c?c+b:c]}),even:nb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:nb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:nb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:nb(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function rb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function sb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function tb(a,b,c){for(var d=0,e=b.length;e>d;d++)fb(a,b[d],c);return c}function ub(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function vb(a,b,c,d,e,f){return d&&!d[u]&&(d=vb(d)),e&&!e[u]&&(e=vb(e,f)),hb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||tb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ub(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ub(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?K.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ub(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):I.apply(g,r)})}function wb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=rb(function(a){return a===b},h,!0),l=rb(function(a){return K.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>i;i++)if(c=d.relative[a[i].type])m=[rb(sb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return vb(i>1&&sb(m),i>1&&qb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&wb(a.slice(i,e)),f>e&&wb(a=a.slice(e)),f>e&&qb(a))}m.push(c)}return sb(m)}function xb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=G.call(i));s=ub(s)}I.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&fb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?hb(f):f}return h=fb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xb(e,d)),f.selector=a}return f},i=fb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&ob(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qb(j),!a)return I.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&ob(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ib(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ib(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||jb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ib(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||jb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ib(function(a){return null==a.getAttribute("disabled")})||jb(L,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fb}(a);m.find=s,m.expr=s.selectors,m.expr[":"]=m.expr.pseudos,m.unique=s.uniqueSort,m.text=s.getText,m.isXMLDoc=s.isXML,m.contains=s.contains;var t=m.expr.match.needsContext,u=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,v=/^.[^:#\[\.,]*$/;function w(a,b,c){if(m.isFunction(b))return m.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return m.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(v.test(b))return m.filter(b,a,c);b=m.filter(b,a)}return m.grep(a,function(a){return m.inArray(a,b)>=0!==c})}m.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?m.find.matchesSelector(d,a)?[d]:[]:m.find.matches(a,m.grep(b,function(a){return 1===a.nodeType}))},m.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(m(a).filter(function(){for(b=0;e>b;b++)if(m.contains(d[b],this))return!0}));for(b=0;e>b;b++)m.find(a,d[b],c);return c=this.pushStack(e>1?m.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(w(this,a||[],!1))},not:function(a){return this.pushStack(w(this,a||[],!0))},is:function(a){return!!w(this,"string"==typeof a&&t.test(a)?m(a):a||[],!1).length}});var x,y=a.document,z=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=m.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||x).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof m?b[0]:b,m.merge(this,m.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:y,!0)),u.test(c[1])&&m.isPlainObject(b))for(c in b)m.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=y.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return x.find(a);this.length=1,this[0]=d}return this.context=y,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):m.isFunction(a)?"undefined"!=typeof x.ready?x.ready(a):a(m):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),m.makeArray(a,this))};A.prototype=m.fn,x=m(y);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};m.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!m(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),m.fn.extend({has:function(a){var b,c=m(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(m.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=t.test(a)||"string"!=typeof a?m(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&m.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?m.unique(f):f)},index:function(a){return a?"string"==typeof a?m.inArray(this[0],m(a)):m.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(m.unique(m.merge(this.get(),m(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function D(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}m.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return m.dir(a,"parentNode")},parentsUntil:function(a,b,c){return m.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return m.dir(a,"nextSibling")},prevAll:function(a){return m.dir(a,"previousSibling")},nextUntil:function(a,b,c){return m.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return m.dir(a,"previousSibling",c)},siblings:function(a){return m.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return m.sibling(a.firstChild)},contents:function(a){return m.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:m.merge([],a.childNodes)}},function(a,b){m.fn[a]=function(c,d){var e=m.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=m.filter(d,e)),this.length>1&&(C[a]||(e=m.unique(e)),B.test(a)&&(e=e.reverse())),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return m.each(a.match(E)||[],function(a,c){b[c]=!0}),b}m.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):m.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){m.each(b,function(b,c){var d=m.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&m.each(arguments,function(a,c){var d;while((d=m.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?m.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},m.extend({Deferred:function(a){var b=[["resolve","done",m.Callbacks("once memory"),"resolved"],["reject","fail",m.Callbacks("once memory"),"rejected"],["notify","progress",m.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return m.Deferred(function(c){m.each(b,function(b,f){var g=m.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&m.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?m.extend(a,d):d}},e={};return d.pipe=d.then,m.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&m.isFunction(a.promise)?e:0,g=1===f?a:m.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&m.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var H;m.fn.ready=function(a){return m.ready.promise().done(a),this},m.extend({isReady:!1,readyWait:1,holdReady:function(a){a?m.readyWait++:m.ready(!0)},ready:function(a){if(a===!0?!--m.readyWait:!m.isReady){if(!y.body)return setTimeout(m.ready);m.isReady=!0,a!==!0&&--m.readyWait>0||(H.resolveWith(y,[m]),m.fn.triggerHandler&&(m(y).triggerHandler("ready"),m(y).off("ready")))}}});function I(){y.addEventListener?(y.removeEventListener("DOMContentLoaded",J,!1),a.removeEventListener("load",J,!1)):(y.detachEvent("onreadystatechange",J),a.detachEvent("onload",J))}function J(){(y.addEventListener||"load"===event.type||"complete"===y.readyState)&&(I(),m.ready())}m.ready.promise=function(b){if(!H)if(H=m.Deferred(),"complete"===y.readyState)setTimeout(m.ready);else if(y.addEventListener)y.addEventListener("DOMContentLoaded",J,!1),a.addEventListener("load",J,!1);else{y.attachEvent("onreadystatechange",J),a.attachEvent("onload",J);var c=!1;try{c=null==a.frameElement&&y.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!m.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}I(),m.ready()}}()}return H.promise(b)};var K="undefined",L;for(L in m(k))break;k.ownLast="0"!==L,k.inlineBlockNeedsLayout=!1,m(function(){var a,b,c,d;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",k.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(d))}),function(){var a=y.createElement("div");if(null==k.deleteExpando){k.deleteExpando=!0;try{delete a.test}catch(b){k.deleteExpando=!1}}a=null}(),m.acceptData=function(a){var b=m.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var M=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,N=/([A-Z])/g;function O(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(N,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:M.test(c)?m.parseJSON(c):c}catch(e){}m.data(a,b,c)}else c=void 0}return c}function P(a){var b;for(b in a)if(("data"!==b||!m.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function Q(a,b,d,e){if(m.acceptData(a)){var f,g,h=m.expando,i=a.nodeType,j=i?m.cache:a,k=i?a[h]:a[h]&&h; +if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){m.isArray(b)?b=b.concat(m.map(b,m.camelCase)):b in d?b=[b]:(b=m.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!P(d):!m.isEmptyObject(d))return}(c||(delete g[h].data,P(g[h])))&&(f?m.cleanData([a],!0):k.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}m.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?m.cache[a[m.expando]]:a[m.expando],!!a&&!P(a)},data:function(a,b,c){return Q(a,b,c)},removeData:function(a,b){return R(a,b)},_data:function(a,b,c){return Q(a,b,c,!0)},_removeData:function(a,b){return R(a,b,!0)}}),m.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=m.data(f),1===f.nodeType&&!m._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=m.camelCase(d.slice(5)),O(f,d,e[d])));m._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){m.data(this,a)}):arguments.length>1?this.each(function(){m.data(this,a,b)}):f?O(f,a,m.data(f,a)):void 0},removeData:function(a){return this.each(function(){m.removeData(this,a)})}}),m.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=m._data(a,b),c&&(!d||m.isArray(c)?d=m._data(a,b,m.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=m.queue(a,b),d=c.length,e=c.shift(),f=m._queueHooks(a,b),g=function(){m.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return m._data(a,c)||m._data(a,c,{empty:m.Callbacks("once memory").add(function(){m._removeData(a,b+"queue"),m._removeData(a,c)})})}}),m.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},W=/^(?:checkbox|radio)$/i;!function(){var a=y.createElement("input"),b=y.createElement("div"),c=y.createDocumentFragment();if(b.innerHTML="
a",k.leadingWhitespace=3===b.firstChild.nodeType,k.tbody=!b.getElementsByTagName("tbody").length,k.htmlSerialize=!!b.getElementsByTagName("link").length,k.html5Clone="<:nav>"!==y.createElement("nav").cloneNode(!0).outerHTML,a.type="checkbox",a.checked=!0,c.appendChild(a),k.appendChecked=a.checked,b.innerHTML="",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,c.appendChild(b),b.innerHTML="",k.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,k.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){k.noCloneEvent=!1}),b.cloneNode(!0).click()),null==k.deleteExpando){k.deleteExpando=!0;try{delete b.test}catch(d){k.deleteExpando=!1}}}(),function(){var b,c,d=y.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(k[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),k[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var X=/^(?:input|select|textarea)$/i,Y=/^key/,Z=/^(?:mouse|pointer|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=/^([^.]*)(?:\.(.+)|)$/;function ab(){return!0}function bb(){return!1}function cb(){try{return y.activeElement}catch(a){}}m.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=m.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof m===K||a&&m.event.triggered===a.type?void 0:m.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(E)||[""],h=b.length;while(h--)f=_.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=m.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=m.event.special[o]||{},l=m.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&m.expr.match.needsContext.test(e),namespace:p.join(".")},i),(n=g[o])||(n=g[o]=[],n.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?n.splice(n.delegateCount++,0,l):n.push(l),m.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m.hasData(a)&&m._data(a);if(r&&(k=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=_.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=m.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,n=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=n.length;while(f--)g=n[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(n.splice(f,1),g.selector&&n.delegateCount--,l.remove&&l.remove.call(a,g));i&&!n.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||m.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)m.event.remove(a,o+b[j],c,d,!0);m.isEmptyObject(k)&&(delete r.handle,m._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,n,o=[d||y],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||y,3!==d.nodeType&&8!==d.nodeType&&!$.test(p+m.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[m.expando]?b:new m.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:m.makeArray(c,[b]),k=m.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!m.isWindow(d)){for(i=k.delegateType||p,$.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||y)&&o.push(l.defaultView||l.parentWindow||a)}n=0;while((h=o[n++])&&!b.isPropagationStopped())b.type=n>1?i:k.bindType||p,f=(m._data(h,"events")||{})[b.type]&&m._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&m.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&m.acceptData(d)&&g&&d[p]&&!m.isWindow(d)){l=d[g],l&&(d[g]=null),m.event.triggered=p;try{d[p]()}catch(r){}m.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=m.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(m._data(this,"events")||{})[a.type]||[],k=m.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=m.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((m.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?m(c,this).index(i)>=0:m.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),hb=/^\s+/,ib=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,jb=/<([\w:]+)/,kb=/\s*$/g,rb={option:[1,""],legend:[1,"
","
"],area:[1,"",""],param:[1,"",""],thead:[1,"","
"],tr:[2,"","
"],col:[2,"","
"],td:[3,"","
"],_default:k.htmlSerialize?[0,"",""]:[1,"X
","
"]},sb=db(y),tb=sb.appendChild(y.createElement("div"));rb.optgroup=rb.option,rb.tbody=rb.tfoot=rb.colgroup=rb.caption=rb.thead,rb.th=rb.td;function ub(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==K?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==K?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||m.nodeName(d,b)?f.push(d):m.merge(f,ub(d,b));return void 0===b||b&&m.nodeName(a,b)?m.merge([a],f):f}function vb(a){W.test(a.type)&&(a.defaultChecked=a.checked)}function wb(a,b){return m.nodeName(a,"table")&&m.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function xb(a){return a.type=(null!==m.find.attr(a,"type"))+"/"+a.type,a}function yb(a){var b=pb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function zb(a,b){for(var c,d=0;null!=(c=a[d]);d++)m._data(c,"globalEval",!b||m._data(b[d],"globalEval"))}function Ab(a,b){if(1===b.nodeType&&m.hasData(a)){var c,d,e,f=m._data(a),g=m._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)m.event.add(b,c,h[c][d])}g.data&&(g.data=m.extend({},g.data))}}function Bb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!k.noCloneEvent&&b[m.expando]){e=m._data(b);for(d in e.events)m.removeEvent(b,d,e.handle);b.removeAttribute(m.expando)}"script"===c&&b.text!==a.text?(xb(b).text=a.text,yb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),k.html5Clone&&a.innerHTML&&!m.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&W.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}m.extend({clone:function(a,b,c){var d,e,f,g,h,i=m.contains(a.ownerDocument,a);if(k.html5Clone||m.isXMLDoc(a)||!gb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(tb.innerHTML=a.outerHTML,tb.removeChild(f=tb.firstChild)),!(k.noCloneEvent&&k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||m.isXMLDoc(a)))for(d=ub(f),h=ub(a),g=0;null!=(e=h[g]);++g)d[g]&&Bb(e,d[g]);if(b)if(c)for(h=h||ub(a),d=d||ub(f),g=0;null!=(e=h[g]);g++)Ab(e,d[g]);else Ab(a,f);return d=ub(f,"script"),d.length>0&&zb(d,!i&&ub(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,l,n=a.length,o=db(b),p=[],q=0;n>q;q++)if(f=a[q],f||0===f)if("object"===m.type(f))m.merge(p,f.nodeType?[f]:f);else if(lb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(jb.exec(f)||["",""])[1].toLowerCase(),l=rb[i]||rb._default,h.innerHTML=l[1]+f.replace(ib,"<$1>")+l[2],e=l[0];while(e--)h=h.lastChild;if(!k.leadingWhitespace&&hb.test(f)&&p.push(b.createTextNode(hb.exec(f)[0])),!k.tbody){f="table"!==i||kb.test(f)?""!==l[1]||kb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)m.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}m.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),k.appendChecked||m.grep(ub(p,"input"),vb),q=0;while(f=p[q++])if((!d||-1===m.inArray(f,d))&&(g=m.contains(f.ownerDocument,f),h=ub(o.appendChild(f),"script"),g&&zb(h),c)){e=0;while(f=h[e++])ob.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=m.expando,j=m.cache,l=k.deleteExpando,n=m.event.special;null!=(d=a[h]);h++)if((b||m.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)n[e]?m.event.remove(d,e):m.removeEvent(d,e,g.handle);j[f]&&(delete j[f],l?delete d[i]:typeof d.removeAttribute!==K?d.removeAttribute(i):d[i]=null,c.push(f))}}}),m.fn.extend({text:function(a){return V(this,function(a){return void 0===a?m.text(this):this.empty().append((this[0]&&this[0].ownerDocument||y).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?m.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||m.cleanData(ub(c)),c.parentNode&&(b&&m.contains(c.ownerDocument,c)&&zb(ub(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&m.cleanData(ub(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&m.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return m.clone(this,a,b)})},html:function(a){return V(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(fb,""):void 0;if(!("string"!=typeof a||mb.test(a)||!k.htmlSerialize&&gb.test(a)||!k.leadingWhitespace&&hb.test(a)||rb[(jb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(ib,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(m.cleanData(ub(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,m.cleanData(ub(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,l=this.length,n=this,o=l-1,p=a[0],q=m.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&nb.test(p))return this.each(function(c){var d=n.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(i=m.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=m.map(ub(i,"script"),xb),f=g.length;l>j;j++)d=i,j!==o&&(d=m.clone(d,!0,!0),f&&m.merge(g,ub(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,m.map(g,yb),j=0;f>j;j++)d=g[j],ob.test(d.type||"")&&!m._data(d,"globalEval")&&m.contains(h,d)&&(d.src?m._evalUrl&&m._evalUrl(d.src):m.globalEval((d.text||d.textContent||d.innerHTML||"").replace(qb,"")));i=c=null}return this}}),m.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){m.fn[a]=function(a){for(var c,d=0,e=[],g=m(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),m(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Cb,Db={};function Eb(b,c){var d,e=m(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:m.css(e[0],"display");return e.detach(),f}function Fb(a){var b=y,c=Db[a];return c||(c=Eb(a,b),"none"!==c&&c||(Cb=(Cb||m("