From 4f7d547b85ed89ba4706e05d7d0984f16749120e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 25 Mar 2014 19:01:18 -0700 Subject: [PATCH 01/27] Initial experimentation with Travis CI configuration This is not intended to replace Jenkins immediately, and Jenkins will remain the CI of reference for merging pull requests in the near term. Long term, it is possible that Travis will give us better integration with github, so we are investigating its use. Author: Michael Armbrust Closes #230 from marmbrus/travis and squashes the following commits: 93f9a32 [Michael Armbrust] Add Apache license to .travis.yml d7c0e78 [Michael Armbrust] Initial experimentation with Travis CI configuration --- .travis.yml | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 0000000000000..617da2dc308cc --- /dev/null +++ b/.travis.yml @@ -0,0 +1,37 @@ +# 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. + + language: scala + scala: + - "2.10.3" + jdk: + - oraclejdk7 + env: + matrix: + - TEST=sql/test + - TEST=hive/test + - TEST=catalyst/test + - TEST=streaming/test + - TEST=graphx/test + - TEST=mllib/test + - TEST=graphx/test + - TEST=bagel/test + cache: + directories: + - $HOME/.m2 + - $HOME/.ivy2 + - $HOME/.sbt + script: + - "sbt ++$TRAVIS_SCALA_VERSION scalastyle $TEST" From b859853ba47b6323af0e31a4e2099e943221e1b1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 26 Mar 2014 00:09:44 -0700 Subject: [PATCH 02/27] SPARK-1321 Use Guava's top k implementation rather than our BoundedPriorityQueue based implementation Also updated the documentation for top and takeOrdered. On my simple test of sorting 100 million (Int, Int) tuples using Spark, Guava's top k implementation (in Ordering) is much faster than the BoundedPriorityQueue implementation for roughly sorted input (10 - 20X faster), and still faster for purely random input (2 - 5X). Author: Reynold Xin Closes #229 from rxin/takeOrdered and squashes the following commits: 0d11844 [Reynold Xin] Use Guava's top k implementation rather than our BoundedPriorityQueue based implementation. Also updated the documentation for top and takeOrdered. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 49 +++++++++++++------ .../apache/spark/util/collection/Utils.scala | 39 +++++++++++++++ 2 files changed, 72 insertions(+), 16 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/Utils.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 4f9d39f86546c..6af42248a5c3c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -927,32 +927,49 @@ abstract class RDD[T: ClassTag]( } /** - * Returns the top K elements from this RDD as defined by - * the specified implicit Ordering[T]. + * Returns the top K (largest) elements from this RDD as defined by the specified + * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For example: + * {{{ + * sc.parallelize([10, 4, 2, 12, 3]).top(1) + * // returns [12] + * + * sc.parallelize([2, 3, 4, 5, 6]).top(2) + * // returns [6, 5] + * }}} + * * @param num the number of top elements to return * @param ord the implicit ordering for T * @return an array of top elements */ - def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { - mapPartitions { items => - val queue = new BoundedPriorityQueue[T](num) - queue ++= items - Iterator.single(queue) - }.reduce { (queue1, queue2) => - queue1 ++= queue2 - queue1 - }.toArray.sorted(ord.reverse) - } + def top(num: Int)(implicit ord: Ordering[T]): Array[T] = takeOrdered(num)(ord.reverse) /** - * Returns the first K elements from this RDD as defined by - * the specified implicit Ordering[T] and maintains the - * ordering. + * Returns the first K (smallest) elements from this RDD as defined by the specified + * implicit Ordering[T] and maintains the ordering. This does the opposite of [[top]]. + * For example: + * {{{ + * sc.parallelize([10, 4, 2, 12, 3]).takeOrdered(1) + * // returns [12] + * + * sc.parallelize([2, 3, 4, 5, 6]).takeOrdered(2) + * // returns [2, 3] + * }}} + * * @param num the number of top elements to return * @param ord the implicit ordering for T * @return an array of top elements */ - def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { + mapPartitions { items => + // Priority keeps the largest elements, so let's reverse the ordering. + val queue = new BoundedPriorityQueue[T](num)(ord.reverse) + queue ++= util.collection.Utils.takeOrdered(items, num)(ord) + Iterator.single(queue) + }.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord) + } /** * Returns the max of this RDD as defined by the implicit Ordering[T]. diff --git a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala new file mode 100644 index 0000000000000..c5268c0fae0ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.collection.JavaConversions.{collectionAsScalaIterable, asJavaIterator} + +import com.google.common.collect.{Ordering => GuavaOrdering} + +/** + * Utility functions for collections. + */ +private[spark] object Utils { + + /** + * Returns the first K elements from the input as defined by the specified implicit Ordering[T] + * and maintains the ordering. + */ + def takeOrdered[T](input: Iterator[T], num: Int)(implicit ord: Ordering[T]): Iterator[T] = { + val ordering = new GuavaOrdering[T] { + override def compare(l: T, r: T) = ord.compare(l, r) + } + collectionAsScalaIterable(ordering.leastOf(asJavaIterator(input), num)).iterator + } +} From a0853a39e1907700737ec924367cd2610da8395a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 26 Mar 2014 09:16:37 -0700 Subject: [PATCH 03/27] SPARK-1322, top in pyspark should sort result in descending order. Author: Prashant Sharma Closes #235 from ScrapCodes/SPARK-1322/top-rev-sort and squashes the following commits: f316266 [Prashant Sharma] Minor change in comment. 58e58c6 [Prashant Sharma] SPARK-1322, top in pyspark should sort result in descending order. --- python/pyspark/rdd.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index ca2dc119a461a..019c249699c2d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -693,11 +693,11 @@ def top(self, num): """ Get the top N elements from a RDD. - Note: It returns the list sorted in ascending order. + Note: It returns the list sorted in descending order. >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) [12] >>> sc.parallelize([2, 3, 4, 5, 6]).cache().top(2) - [5, 6] + [6, 5] """ def topIterator(iterator): q = [] @@ -711,7 +711,7 @@ def topIterator(iterator): def merge(a, b): return next(topIterator(a + b)) - return sorted(self.mapPartitions(topIterator).reduce(merge)) + return sorted(self.mapPartitions(topIterator).reduce(merge), reverse=True) def take(self, num): """ From 345825d97987b9eeb2afcf002f815a05ff51fc2e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 26 Mar 2014 15:36:18 -0700 Subject: [PATCH 04/27] Unified package definition format in Spark SQL According to discussions in comments of PR #208, this PR unifies package definition format in Spark SQL. Some broken links in ScalaDoc and typos detected along the way are also fixed. Author: Cheng Lian Closes #225 from liancheng/packageDefinition and squashes the following commits: 75c47b3 [Cheng Lian] Fixed file line length 4f87968 [Cheng Lian] Unified package definition format in Spark SQL --- .../scala/org/apache/spark/Dependency.scala | 5 ++- .../apache/spark/api/java/JavaRDDLike.scala | 4 +- .../spark/sql/catalyst/ScalaReflection.scala | 3 +- .../sql/catalyst/analysis/Analyzer.scala | 9 ++--- .../spark/sql/catalyst/analysis/Catalog.scala | 4 +- .../catalyst/analysis/FunctionRegistry.scala | 4 +- .../catalyst/analysis/HiveTypeCoercion.scala | 8 ++-- .../analysis/MultiInstanceRelation.scala | 3 +- .../spark/sql/catalyst/analysis/package.scala | 3 +- .../sql/catalyst/analysis/unresolved.scala | 5 +-- .../spark/sql/catalyst/dsl/package.scala | 3 +- .../spark/sql/catalyst/errors/package.scala | 3 +- .../catalyst/expressions/BoundAttribute.scala | 6 +-- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 9 ++--- .../sql/catalyst/expressions/Projection.scala | 3 +- .../spark/sql/catalyst/expressions/Rand.scala | 4 +- .../spark/sql/catalyst/expressions/Row.scala | 4 +- .../sql/catalyst/expressions/ScalaUdf.scala | 4 +- .../sql/catalyst/expressions/SortOrder.scala | 4 +- .../catalyst/expressions/WrapDynamic.scala | 4 +- .../sql/catalyst/expressions/aggregates.scala | 5 +-- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../catalyst/expressions/complexTypes.scala | 4 +- .../sql/catalyst/expressions/generators.scala | 5 +-- .../sql/catalyst/expressions/literals.scala | 4 +- .../expressions/namedExpressions.scala | 5 +-- .../catalyst/expressions/nullFunctions.scala | 5 +-- .../sql/catalyst/expressions/package.scala | 5 +-- .../sql/catalyst/expressions/predicates.scala | 7 ++-- .../expressions/stringOperations.scala | 4 +- .../sql/catalyst/optimizer/Optimizer.scala | 5 +-- .../sql/catalyst/planning/QueryPlanner.scala | 5 +-- .../spark/sql/catalyst/planning/package.scala | 3 +- .../sql/catalyst/planning/patterns.scala | 8 ++-- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../spark/sql/catalyst/plans/joinTypes.scala | 4 +- .../catalyst/plans/logical/BaseRelation.scala | 5 +-- .../catalyst/plans/logical/LogicalPlan.scala | 7 ++-- .../plans/logical/ScriptTransformation.scala | 5 +-- .../catalyst/plans/logical/TestRelation.scala | 6 +-- .../plans/logical/basicOperators.scala | 6 +-- .../catalyst/plans/logical/partitioning.scala | 5 +-- .../spark/sql/catalyst/plans/package.scala | 3 +- .../plans/physical/partitioning.scala | 5 +-- .../spark/sql/catalyst/rules/Rule.scala | 5 +-- .../spark/sql/catalyst/rules/package.scala | 3 +- .../spark/sql/catalyst/trees/TreeNode.scala | 4 +- .../spark/sql/catalyst/trees/package.scala | 5 ++- .../spark/sql/catalyst/types/dataTypes.scala | 4 +- .../spark/sql/catalyst/types/package.scala | 3 +- .../spark/sql/catalyst/util/package.scala | 3 +- .../sql/catalyst/DistributionSuite.scala | 5 +-- .../sql/catalyst/analysis/AnalysisSuite.scala | 4 +- .../analysis/HiveTypeCoercionSuite.scala | 4 +- .../ExpressionEvaluationSuite.scala | 4 +- .../optimizer/ConstantFoldingSuite.scala | 6 +-- .../optimizer/FilterPushdownSuite.scala | 8 ++-- .../catalyst/optimizer/OptimizerTest.scala | 4 +- .../catalyst/trees/RuleExecutorSuite.scala | 4 +- .../spark/sql/columnar/ColumnAccessor.scala | 3 +- .../spark/sql/columnar/ColumnBuilder.scala | 8 ++-- .../spark/sql/columnar/ColumnType.scala | 4 +- .../sql/columnar/NullableColumnBuilder.scala | 7 ++-- .../columnar/inMemoryColumnarOperators.scala | 10 ++--- .../apache/spark/sql/execution/Exchange.scala | 6 +-- .../apache/spark/sql/execution/Generate.scala | 5 +-- .../execution/QueryExecutionException.scala | 3 +- .../spark/sql/execution/SparkPlan.scala | 13 +++---- .../sql/execution/SparkSqlSerializer.scala | 3 +- .../spark/sql/execution/SparkStrategies.scala | 6 +-- .../spark/sql/execution/aggregates.scala | 3 +- .../spark/sql/execution/basicOperators.scala | 3 +- .../apache/spark/sql/execution/debug.scala | 3 +- .../apache/spark/sql/execution/joins.scala | 3 +- .../spark/sql/parquet/ParquetRelation.scala | 9 ++--- .../sql/parquet/ParquetTableSupport.scala | 3 +- .../spark/sql/test/TestSQLContext.scala | 7 ++-- .../org/apache/spark/sql/QueryTest.scala | 2 - .../spark/sql/columnar/ColumnTypeSuite.scala | 3 +- .../NullableColumnAccessorSuite.scala | 3 +- .../columnar/NullableColumnBuilderSuite.scala | 3 +- .../apache/spark/sql/execution/TgfSuite.scala | 3 +- .../org/apache/spark/SparkHadoopWriter.scala | 13 +++---- .../apache/spark/sql/hive/HiveContext.scala | 38 +++++++++---------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 ++-- .../org/apache/spark/sql/hive/HiveQl.scala | 3 +- .../spark/sql/hive/HiveStrategies.scala | 4 +- .../spark/sql/hive/ScriptTransformation.scala | 3 +- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/TestHive.scala | 3 +- .../apache/spark/sql/hive/hiveOperators.scala | 3 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 4 +- .../execution/BigDataBenchmarkSuite.scala | 10 ++--- .../hive/execution/ConcurrentHiveSuite.scala | 9 ++--- .../hive/execution/HiveComparisonTest.scala | 9 ++--- .../execution/HiveCompatibilitySuite.scala | 6 +-- .../hive/execution/HiveQueryFileTest.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 8 ++-- .../hive/execution/HiveResolutionSuite.scala | 7 ++-- .../sql/hive/execution/HiveSerDeSuite.scala | 4 +- .../execution/HiveTypeCoercionSuite.scala | 3 +- .../sql/hive/execution/PruningSuite.scala | 6 +-- 103 files changed, 209 insertions(+), 333 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 448f87b81ef4a..3132dcf745e19 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -44,8 +44,9 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output - * @param serializer [[Serializer]] to use. If set to null, the default serializer, as specified - * by `spark.serializer` config option, will be used. + * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null, + * the default serializer, as specified by `spark.serializer` config option, will + * be used. */ class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 05b89b985736d..ddac553304233 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -481,7 +481,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Returns the maximum element from this RDD as defined by the specified * Comparator[T]. - * @params comp the comparator that defines ordering + * @param comp the comparator that defines ordering * @return the maximum of the RDD * */ def max(comp: Comparator[T]): T = { @@ -491,7 +491,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Returns the minimum element from this RDD as defined by the specified * Comparator[T]. - * @params comp the comparator that defines ordering + * @param comp the comparator that defines ordering * @return the minimum of the RDD * */ def min(comp: Comparator[T]): T = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index bf7318d2e078b..976dda8d7e59a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.AttributeReference diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 161d28eba070e..4ebc0e70d946b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -89,7 +87,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool /** * Replaces [[UnresolvedAttribute]]s with concrete - * [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children. + * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's + * children. */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { @@ -106,7 +105,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]]. + * Replaces [[UnresolvedFunction]]s with concrete [[catalyst.expressions.Expression Expressions]]. */ object ResolveFunctions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { 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 b77f0bbb2f21b..ff66177a03b8c 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index eed058d3c7482..c0255701b7ba5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Expression diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index a6ecf6e2ebec5..4557d77160fad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} @@ -37,8 +35,8 @@ trait HiveTypeCoercion { StringToIntegralCasts, FunctionArgumentConversion) /** - * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes - * that are made by other rules to instances higher in the query tree. + * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] data + * types that are made by other rules to instances higher in the query tree. */ object PropagateTypes extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index 3cad3a5d4db4e..a6ce90854dcb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 30c55bacc7427..9f37ca904ffeb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 04ae481102c74..41e9bcef3cd7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.{errors, trees} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index e6255bcafa8aa..67cddb351c185 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import scala.language.implicitConversions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala index d8b69946fab10..bdeb660b1ecb7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 3fa4148f0dff2..f70e80b7f27f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.Logging /** * A bound reference points to a specific slot in the input tuple, allowing the actual value diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 71f64ef95064e..c26fc3d0f305f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 2454a3355b11f..81fd160e00ca1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType} -import org.apache.spark.sql.catalyst.errors.TreeNodeException abstract class Expression extends TreeNode[Expression] { self: Product => @@ -69,7 +68,7 @@ abstract class Expression extends TreeNode[Expression] { def childrenResolved = !children.exists(!_.resolved) /** - * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type + * A set of helper functions that return the correct descendant of `scala.math.Numeric[T]` type * and do any casting necessary of child evaluation. */ @inline diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 8c407d2fddb42..38542d3fc7290 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions /** * Converts a [[Row]] to another Row given a sequence of expression that define each column of the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index 0d173afec8901..0bde621602944 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 79c91ebaa4746..31d42b9ee71a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.NativeType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index cc339480550a7..f53d8504b083f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.DataType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index 171997b90ecf4..d5d93778f4b8d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions abstract sealed class SortDirection case object Ascending extends SortDirection diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index 01b7a14d4aca6..9828d0b9bd8b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import scala.language.dynamics diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index a16bb80df37af..7303b155cae3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.trees abstract class AggregateExpression extends Expression { self: Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 81e4a487bdbf9..fba056e7c07e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 9ec0f6ade7467..ab96618d73df7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 9097c635ee96f..e9b491b10a5f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.types._ /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 0d01312c71c70..b82a12e0f754e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 47b1241e714af..69c8bed309c18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 38e38371b1dec..5a47768dcb4a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException case class Coalesce(children: Seq[Expression]) extends Expression { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 76554e160b1d1..573ec052f4266 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * A set of classes that can be used to represent trees of relational expressions. A key goal of @@ -46,6 +45,6 @@ package catalyst * ability to reason about which subquery produced a given attribute. * * ==Evaluation== - * The result of expressions can be evaluated using the [[Evaluate]] object. + * The result of expressions can be evaluated using the `Expression.apply(Row)` method. */ package object expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index e7f3e8ca60a25..722ff517d250e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.types.{BooleanType, StringType} +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.types.{BooleanType, StringType} trait Predicate extends Expression { self: Product => 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 7584fe03cf745..e195f2ac7efd1 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.BooleanType 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 f28076999ddbf..3dd6818029bcf 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner @@ -125,7 +123,6 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { grandChild)) } - // def replaceAlias(condition: Expression, sourceAliases: Map[Attribute, Expression]): Expression = { condition transform { case a: AttributeReference => sourceAliases.getOrElse(a, a) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index d50b963dfc25b..67833664b35ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package planning +package org.apache.spark.sql.catalyst.planning +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala index 64370ec7c0cc4..0a030b6c21f1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * Contains classes for enumerating possible physical plans for a given logical query plan. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index ff0ea90e54842..6dd816aa91dd1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package planning +package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec @@ -48,7 +46,9 @@ object FilteredOperation extends PredicateHelper { /** * A pattern that matches any number of project or filter operations on top of another relational * operator. All filter operators are collected and their conditions are broken up and returned - * together with the top project operator. [[Alias Aliases]] are in-lined/substituted if necessary. + * together with the top project operator. + * [[org.apache.spark.sql.catalyst.expressions.Alias Aliases]] are in-lined/substituted if + * necessary. */ object PhysicalOperation extends PredicateHelper { type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 848db2452ae9c..8199a80f5d6bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans +package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 9f2283ad4346e..ae8d7d3e4257f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans +package org.apache.spark.sql.catalyst.plans sealed abstract class JoinType case object Inner extends JoinType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala index 48ff45c3d3ebb..7c616788a3830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical abstract class BaseRelation extends LeafNode { self: Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 225dd260fb9de..5eb52d5350f55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.catalyst.trees abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 5a3ea9f0a0464..d3f9d0fb93237 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala index ac7d2d6001779..f8fe558511bfd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Attribute object LocalRelation { 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 61481de65e76e..9d16189deedfe 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 @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 775e50bbd5128..7146fbd540f29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index a40ab4bbb154b..42bdab42b79ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * A a collection of common abstractions for query plans as well as diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 20e2a45678c89..8893744eb2e7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package physical +package org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} import org.apache.spark.sql.catalyst.types.IntegerType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index c7632a62a00d2..1076537bc7602 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package rules +package org.apache.spark.sql.catalyst.rules +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala index 26ab5430821f4..ca82c5d93d4cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * A framework for applying batches rewrite rules to trees, possibly to fixed point. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 89e27d81dad0b..0369129393a08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package trees +package org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index e2da1d2439bbf..d159ecdd5d781 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst + +import org.apache.spark.sql.Logger /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 90a9f9f7e548d..7a45d1a1b8195 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package types +package org.apache.spark.sql.catalyst.types import scala.reflect.runtime.universe.{typeTag, TypeTag} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala index b65a5617d9a7f..de24449590f9a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * Contains a type system for attributes produced by relations, including complex types like * structs, arrays and maps. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 52adea2661363..a001d953592db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index fb25e1c246117..46b2250aab231 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import org.scalatest.FunSuite @@ -172,4 +171,4 @@ class DistributionSuite extends FunSuite { AllTuples, false) } -} \ No newline at end of file +} 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 78ec48ba77628..4c313585c6386 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index b85b72a2840f9..b9e0f8e9dcc5f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index c8fd581aa7b47..94894adf81202 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 53f760fb4ceb2..2ab14f48ccc8a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index ae1b2b13dd8f1..ef47850455a37 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer - -import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala index 0c6e4d5acee83..89982d5cd8d74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala index 738cfa85fbc6a..4b2d45584045f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package trees +package org.apache.spark.sql.catalyst.trees import org.scalatest.FunSuite diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index ddbeba6203aa4..e0c98ecdf8f22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import java.nio.{ByteOrder, ByteBuffer} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 6bd1841821875..3e622adfd3d6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.columnar.ColumnBuilder._ import org.apache.spark.sql.execution.SparkSqlSerializer private[sql] trait ColumnBuilder { @@ -35,7 +36,6 @@ private[sql] trait ColumnBuilder { } private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder { - import ColumnBuilder._ private var columnName: String = _ protected var buffer: ByteBuffer = _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 3b759a51cc695..a452b86f0cda3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -14,8 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql -package columnar + +package org.apache.spark.sql.columnar import java.nio.ByteBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index 1661c3f3ff4a9..048d1f05c7df2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} + +import org.apache.spark.sql.Row /** * Builds a nullable column. The byte buffer of a nullable column contains: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala index c7efd30e87da4..f853759e5a306 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} import org.apache.spark.sql.execution.{SparkPlan, LeafNode} +import org.apache.spark.sql.Row + +/* Implicit conversions */ +import org.apache.spark.sql.columnar.ColumnType._ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan) extends LeafNode { - // For implicit conversion from `DataType` to `ColumnType` - import ColumnType._ - override def output: Seq[Attribute] = attributes lazy val cachedColumnBuffers = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 65d77e3a40cf4..869673b1fe978 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution +import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions.{MutableProjection, RowOrdering} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair -import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 7e50fda4ceac5..e902e6ced521d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection} /** * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala index 7ce8608d2061d..16806c620635f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala @@ -15,7 +15,6 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution class QueryExecutionException(message: String) extends Exception(message) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 5626181d184e5..acb1ee83a72f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD - +import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.trees abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => @@ -47,7 +46,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { def executeCollect(): Array[Row] = execute().collect() protected def buildRow(values: Seq[Any]): Row = - new catalyst.expressions.GenericRow(values.toArray) + new GenericRow(values.toArray) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index ad7cd58b6aaaf..1c3196ae2e7b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import java.nio.ByteBuffer 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 8a39ded0a9ec4..86f9d3e0fa954 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 @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution +import org.apache.spark.sql.{SQLContext, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ @@ -175,7 +175,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => InsertIntoParquetTable(table, planLater(child))(sparkContext) :: Nil - case PhysicalOperation(projectList, filters, relation: parquet.ParquetRelation) => + case PhysicalOperation(projectList, filters, relation: ParquetRelation) => // TODO: Should be pushing down filters as well. pruneFilterProject( projectList, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala index 14e5ab628f23d..8515a18f18c55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ 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 e4f918b6787fe..65cb8f8becefa 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import scala.reflect.runtime.universe.TypeTag diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala index db259b4c4b95b..40982f1fffbbf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution object DebugQuery { def apply(plan: SparkPlan): SparkPlan = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index a6e3892e88703..f0d21143ba5d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import scala.collection.mutable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 011aaf7440cb1..2b825f84ee910 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -82,11 +82,10 @@ object ParquetRelation { type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow /** - * Creates a new ParquetRelation and underlying Parquetfile for the given - * LogicalPlan. Note that this is used inside [[SparkStrategies]] to - * create a resolved relation as a data sink for writing to a Parquetfile. - * The relation is empty but is initialized with ParquetMetadata and - * can be inserted into. + * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that + * this is used inside [[org.apache.spark.sql.execution.SparkStrategies SparkStrategies]] to + * create a resolved relation as a data sink for writing to a Parquetfile. The relation is empty + * but is initialized with ParquetMetadata and can be inserted into. * * @param pathString The directory the Parquetfile will be stored in. * @param child The child node that will be used for extracting the schema. 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 91b4848fe403f..c21e400282004 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 @@ -135,8 +135,7 @@ object RowWriteSupport { } /** - * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record - * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record to a `Row` object. * * @param schema The corresponding Catalyst schema in the form of a list of attributes. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index ca56c4476bcc2..f2389f8f0591e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark -package sql -package test +package org.apache.spark.sql.test + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SQLContext /** A SQLContext that can be used for local testing. */ object TestSQLContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 5c8cb086ee3b8..d719ceb827691 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -22,8 +22,6 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ -/* Implicits */ - class QueryTest extends FunSuite { /** * Runs the plan and makes sure the answer matches the expected result. 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 1997d8638cb61..2d431affbcfcc 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import java.nio.ByteBuffer diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 279607ccfaa5b..d413d483f4e7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types.DataType 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 3354da3fa3e0f..5222a47e1ab87 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import org.scalatest.FunSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index 93b2a308a410a..ca5c8b8eb63dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index d96c2f70e0c74..7219c030cb0f0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -113,11 +113,10 @@ class SparkHiveHadoopWriter( cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") } catch { - case e: IOException => { + case e: IOException => logError("Error committing the output of task: " + taID.value, e) cmtr.abortTask(taCtxt) throw e - } } } else { logWarning ("No need to commit output of task: " + taID.value) @@ -161,12 +160,12 @@ class SparkHiveHadoopWriter( taskContext } - private def setIDs(jobid: Int, splitid: Int, attemptid: Int) { - jobID = jobid - splitID = splitid - attemptID = attemptid + private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { + jobID = jobId + splitID = splitId + attemptID = attemptId - jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid)) + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) taID = new SerializableWritable[TaskAttemptID]( new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 3bcf586662f2c..fc5057b73fe24 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -32,7 +32,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan, LowerCaseSchema} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand, ExplainCommand} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution._ @@ -86,7 +86,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val input = new java.io.InputStream { val iterator = (start ++ end).iterator - def read(): Int = if (iterator.hasNext) iterator.next else -1 + def read(): Int = if (iterator.hasNext) iterator.next() else -1 } val reader = new BufferedReader(new InputStreamReader(input)) val stringBuilder = new StringBuilder @@ -148,24 +148,24 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { SessionState.start(sessionState) - if (proc.isInstanceOf[Driver]) { - val driver: Driver = proc.asInstanceOf[Driver] - driver.init() + proc match { + case driver: Driver => + driver.init() - val results = new JArrayList[String] - val response: CommandProcessorResponse = driver.run(cmd) - // Throw an exception if there is an error in query processing. - if (response.getResponseCode != 0) { + val results = new JArrayList[String] + val response: CommandProcessorResponse = driver.run(cmd) + // Throw an exception if there is an error in query processing. + if (response.getResponseCode != 0) { + driver.destroy() + throw new QueryExecutionException(response.getErrorMessage) + } + driver.setMaxRows(maxRows) + driver.getResults(results) driver.destroy() - throw new QueryExecutionException(response.getErrorMessage) - } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.destroy() - results - } else { - sessionState.out.println(tokens(0) + " " + cmd_1) - Seq(proc.run(cmd_1).getResponseCode.toString) + results + case _ => + sessionState.out.println(tokens(0) + " " + cmd_1) + Seq(proc.run(cmd_1).getResponseCode.toString) } } catch { case e: Exception => @@ -226,7 +226,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { sparkContext.parallelize(asRows, 1) } case _ => - executedPlan.execute.map(_.copy()) + executedPlan.execute().map(_.copy()) } protected val primitiveTypes = 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 1667a217297b1..4f8353666a12b 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import scala.util.parsing.combinator.RegexParsers @@ -27,14 +26,15 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer - -import org.apache.spark.sql.catalyst.analysis.{Catalog, EliminateAnalysisOperators} +import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ +/* Implicit conversions */ import scala.collection.JavaConversions._ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 8e76a7348e957..f4b61381f9a27 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 141067247d736..3ca1d93c11fa9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala index dc4181ec9905c..2610100043659 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import java.io.{BufferedReader, InputStreamReader} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 99dc85ec19868..ca5311344615f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a26b0ff2319d4..bc3447b9d802d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import java.io.File import java.util.{Set => JavaSet} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index 78f69e7ff5731..e2d9d8de2572a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.metastore.MetaStoreUtils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 72ccd4f4a4302..44901db3f963b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import scala.collection.mutable.ArrayBuffer @@ -29,6 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 4b45e698601eb..8488f23abd0fd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -15,19 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution import java.io.File +import org.apache.spark.sql.hive.TestHive._ + /** * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - import TestHive._ - val testDataDirectory = new File("target/big-data-benchmark-testdata") val testTables = Seq( @@ -123,4 +121,4 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { |-- SELECT * FROM url_counts_total """.stripMargin) } -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index 02ee2a0ebc354..ac87f2cb10d12 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark -package sql -package hive -package execution +package org.apache.spark.sql.hive.execution -import org.scalatest.{FunSuite, BeforeAndAfterAll} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.hive.TestHiveContext +import org.scalatest.{BeforeAndAfterAll, FunSuite} class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { ignore("multiple instances not supported") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index e8fcc272354fa..c7a350ef94edd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -15,17 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution import java.io._ -import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} - +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.Sort +import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} +import org.apache.spark.sql.hive.TestHive /** * Allows the creations of tests that execute the same query against both hive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index d5b54fa90fed3..f74b0fbb97c83 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.hive.TestHive /** * Runs the test cases that are included in the hive distribution. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 2d2f13333a65c..50ab71a9003d3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution import java.io.File 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 b804634db1f5f..4b92d167a1263 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 @@ -15,16 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.hive.TestHive._ /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest { - import TestHive._ - createQueryTest("Simple Average", "SELECT AVG(key) FROM src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 4bdea214677ad..d77900ddc950c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution -import TestHive._ +import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.TestHive._ case class Data(a: Int, B: Int, n: Nested) case class Nested(a: Int, B: Int) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index c2264926f4356..df9bae96494d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution /** * A set of tests that validates support for Hive SerDe. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index bb33583e5f582..e030c8ee3dfc8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -21,7 +21,6 @@ package org.apache.spark.sql.hive.execution * A set of tests that validate type promotion rules. */ class HiveTypeCoercionSuite extends HiveComparisonTest { - val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'") baseTypes.foreach { i => @@ -29,4 +28,4 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1") } } -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index d2f8e5df5b29e..1318ac1968dad 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.{HiveTableScan, TestHive} /* Implicit conversions */ import scala.collection.JavaConversions._ From 32cbdfd2887f7a792f360ac3224f8c38cc97d21f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 26 Mar 2014 18:19:15 -0700 Subject: [PATCH 05/27] [SQL] Un-ignore a test that is now passing. Add golden answer for aforementioned test. Also, fix golden test generation from sbt/sbt by setting the classpath correctly. Author: Michael Armbrust Closes #244 from marmbrus/partTest and squashes the following commits: 37a33c9 [Michael Armbrust] Un-ignore a test that is now passing, add golden answer for aforementioned test. Fix golden test generation from sbt/sbt. --- sbt/sbt | 8 + ...le scan-0-3e8898a13ccef627603f340d1f8bdd80 | 2000 +++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 6 +- 3 files changed, 2010 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 diff --git a/sbt/sbt b/sbt/sbt index 3ffa4ed9ab5a7..9de265bd07dcb 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,5 +1,13 @@ #!/usr/bin/env bash +# When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so +# that we can run Hive to generate the golden answer. This is not required for normal development +# or testing. +for i in $HIVE_HOME/lib/* +do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i +done +export HADOOP_CLASSPATH + realpath () { ( TARGET_FILE=$1 diff --git a/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 b/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 new file mode 100644 index 0000000000000..a3cb00feaca62 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 @@ -0,0 +1,2000 @@ +2008-04-08 11 238 val_238 +2008-04-08 11 86 val_86 +2008-04-08 11 311 val_311 +2008-04-08 11 27 val_27 +2008-04-08 11 165 val_165 +2008-04-08 11 409 val_409 +2008-04-08 11 255 val_255 +2008-04-08 11 278 val_278 +2008-04-08 11 98 val_98 +2008-04-08 11 484 val_484 +2008-04-08 11 265 val_265 +2008-04-08 11 193 val_193 +2008-04-08 11 401 val_401 +2008-04-08 11 150 val_150 +2008-04-08 11 273 val_273 +2008-04-08 11 224 val_224 +2008-04-08 11 369 val_369 +2008-04-08 11 66 val_66 +2008-04-08 11 128 val_128 +2008-04-08 11 213 val_213 +2008-04-08 11 146 val_146 +2008-04-08 11 406 val_406 +2008-04-08 11 429 val_429 +2008-04-08 11 374 val_374 +2008-04-08 11 152 val_152 +2008-04-08 11 469 val_469 +2008-04-08 11 145 val_145 +2008-04-08 11 495 val_495 +2008-04-08 11 37 val_37 +2008-04-08 11 327 val_327 +2008-04-08 11 281 val_281 +2008-04-08 11 277 val_277 +2008-04-08 11 209 val_209 +2008-04-08 11 15 val_15 +2008-04-08 11 82 val_82 +2008-04-08 11 403 val_403 +2008-04-08 11 166 val_166 +2008-04-08 11 417 val_417 +2008-04-08 11 430 val_430 +2008-04-08 11 252 val_252 +2008-04-08 11 292 val_292 +2008-04-08 11 219 val_219 +2008-04-08 11 287 val_287 +2008-04-08 11 153 val_153 +2008-04-08 11 193 val_193 +2008-04-08 11 338 val_338 +2008-04-08 11 446 val_446 +2008-04-08 11 459 val_459 +2008-04-08 11 394 val_394 +2008-04-08 11 237 val_237 +2008-04-08 11 482 val_482 +2008-04-08 11 174 val_174 +2008-04-08 11 413 val_413 +2008-04-08 11 494 val_494 +2008-04-08 11 207 val_207 +2008-04-08 11 199 val_199 +2008-04-08 11 466 val_466 +2008-04-08 11 208 val_208 +2008-04-08 11 174 val_174 +2008-04-08 11 399 val_399 +2008-04-08 11 396 val_396 +2008-04-08 11 247 val_247 +2008-04-08 11 417 val_417 +2008-04-08 11 489 val_489 +2008-04-08 11 162 val_162 +2008-04-08 11 377 val_377 +2008-04-08 11 397 val_397 +2008-04-08 11 309 val_309 +2008-04-08 11 365 val_365 +2008-04-08 11 266 val_266 +2008-04-08 11 439 val_439 +2008-04-08 11 342 val_342 +2008-04-08 11 367 val_367 +2008-04-08 11 325 val_325 +2008-04-08 11 167 val_167 +2008-04-08 11 195 val_195 +2008-04-08 11 475 val_475 +2008-04-08 11 17 val_17 +2008-04-08 11 113 val_113 +2008-04-08 11 155 val_155 +2008-04-08 11 203 val_203 +2008-04-08 11 339 val_339 +2008-04-08 11 0 val_0 +2008-04-08 11 455 val_455 +2008-04-08 11 128 val_128 +2008-04-08 11 311 val_311 +2008-04-08 11 316 val_316 +2008-04-08 11 57 val_57 +2008-04-08 11 302 val_302 +2008-04-08 11 205 val_205 +2008-04-08 11 149 val_149 +2008-04-08 11 438 val_438 +2008-04-08 11 345 val_345 +2008-04-08 11 129 val_129 +2008-04-08 11 170 val_170 +2008-04-08 11 20 val_20 +2008-04-08 11 489 val_489 +2008-04-08 11 157 val_157 +2008-04-08 11 378 val_378 +2008-04-08 11 221 val_221 +2008-04-08 11 92 val_92 +2008-04-08 11 111 val_111 +2008-04-08 11 47 val_47 +2008-04-08 11 72 val_72 +2008-04-08 11 4 val_4 +2008-04-08 11 280 val_280 +2008-04-08 11 35 val_35 +2008-04-08 11 427 val_427 +2008-04-08 11 277 val_277 +2008-04-08 11 208 val_208 +2008-04-08 11 356 val_356 +2008-04-08 11 399 val_399 +2008-04-08 11 169 val_169 +2008-04-08 11 382 val_382 +2008-04-08 11 498 val_498 +2008-04-08 11 125 val_125 +2008-04-08 11 386 val_386 +2008-04-08 11 437 val_437 +2008-04-08 11 469 val_469 +2008-04-08 11 192 val_192 +2008-04-08 11 286 val_286 +2008-04-08 11 187 val_187 +2008-04-08 11 176 val_176 +2008-04-08 11 54 val_54 +2008-04-08 11 459 val_459 +2008-04-08 11 51 val_51 +2008-04-08 11 138 val_138 +2008-04-08 11 103 val_103 +2008-04-08 11 239 val_239 +2008-04-08 11 213 val_213 +2008-04-08 11 216 val_216 +2008-04-08 11 430 val_430 +2008-04-08 11 278 val_278 +2008-04-08 11 176 val_176 +2008-04-08 11 289 val_289 +2008-04-08 11 221 val_221 +2008-04-08 11 65 val_65 +2008-04-08 11 318 val_318 +2008-04-08 11 332 val_332 +2008-04-08 11 311 val_311 +2008-04-08 11 275 val_275 +2008-04-08 11 137 val_137 +2008-04-08 11 241 val_241 +2008-04-08 11 83 val_83 +2008-04-08 11 333 val_333 +2008-04-08 11 180 val_180 +2008-04-08 11 284 val_284 +2008-04-08 11 12 val_12 +2008-04-08 11 230 val_230 +2008-04-08 11 181 val_181 +2008-04-08 11 67 val_67 +2008-04-08 11 260 val_260 +2008-04-08 11 404 val_404 +2008-04-08 11 384 val_384 +2008-04-08 11 489 val_489 +2008-04-08 11 353 val_353 +2008-04-08 11 373 val_373 +2008-04-08 11 272 val_272 +2008-04-08 11 138 val_138 +2008-04-08 11 217 val_217 +2008-04-08 11 84 val_84 +2008-04-08 11 348 val_348 +2008-04-08 11 466 val_466 +2008-04-08 11 58 val_58 +2008-04-08 11 8 val_8 +2008-04-08 11 411 val_411 +2008-04-08 11 230 val_230 +2008-04-08 11 208 val_208 +2008-04-08 11 348 val_348 +2008-04-08 11 24 val_24 +2008-04-08 11 463 val_463 +2008-04-08 11 431 val_431 +2008-04-08 11 179 val_179 +2008-04-08 11 172 val_172 +2008-04-08 11 42 val_42 +2008-04-08 11 129 val_129 +2008-04-08 11 158 val_158 +2008-04-08 11 119 val_119 +2008-04-08 11 496 val_496 +2008-04-08 11 0 val_0 +2008-04-08 11 322 val_322 +2008-04-08 11 197 val_197 +2008-04-08 11 468 val_468 +2008-04-08 11 393 val_393 +2008-04-08 11 454 val_454 +2008-04-08 11 100 val_100 +2008-04-08 11 298 val_298 +2008-04-08 11 199 val_199 +2008-04-08 11 191 val_191 +2008-04-08 11 418 val_418 +2008-04-08 11 96 val_96 +2008-04-08 11 26 val_26 +2008-04-08 11 165 val_165 +2008-04-08 11 327 val_327 +2008-04-08 11 230 val_230 +2008-04-08 11 205 val_205 +2008-04-08 11 120 val_120 +2008-04-08 11 131 val_131 +2008-04-08 11 51 val_51 +2008-04-08 11 404 val_404 +2008-04-08 11 43 val_43 +2008-04-08 11 436 val_436 +2008-04-08 11 156 val_156 +2008-04-08 11 469 val_469 +2008-04-08 11 468 val_468 +2008-04-08 11 308 val_308 +2008-04-08 11 95 val_95 +2008-04-08 11 196 val_196 +2008-04-08 11 288 val_288 +2008-04-08 11 481 val_481 +2008-04-08 11 457 val_457 +2008-04-08 11 98 val_98 +2008-04-08 11 282 val_282 +2008-04-08 11 197 val_197 +2008-04-08 11 187 val_187 +2008-04-08 11 318 val_318 +2008-04-08 11 318 val_318 +2008-04-08 11 409 val_409 +2008-04-08 11 470 val_470 +2008-04-08 11 137 val_137 +2008-04-08 11 369 val_369 +2008-04-08 11 316 val_316 +2008-04-08 11 169 val_169 +2008-04-08 11 413 val_413 +2008-04-08 11 85 val_85 +2008-04-08 11 77 val_77 +2008-04-08 11 0 val_0 +2008-04-08 11 490 val_490 +2008-04-08 11 87 val_87 +2008-04-08 11 364 val_364 +2008-04-08 11 179 val_179 +2008-04-08 11 118 val_118 +2008-04-08 11 134 val_134 +2008-04-08 11 395 val_395 +2008-04-08 11 282 val_282 +2008-04-08 11 138 val_138 +2008-04-08 11 238 val_238 +2008-04-08 11 419 val_419 +2008-04-08 11 15 val_15 +2008-04-08 11 118 val_118 +2008-04-08 11 72 val_72 +2008-04-08 11 90 val_90 +2008-04-08 11 307 val_307 +2008-04-08 11 19 val_19 +2008-04-08 11 435 val_435 +2008-04-08 11 10 val_10 +2008-04-08 11 277 val_277 +2008-04-08 11 273 val_273 +2008-04-08 11 306 val_306 +2008-04-08 11 224 val_224 +2008-04-08 11 309 val_309 +2008-04-08 11 389 val_389 +2008-04-08 11 327 val_327 +2008-04-08 11 242 val_242 +2008-04-08 11 369 val_369 +2008-04-08 11 392 val_392 +2008-04-08 11 272 val_272 +2008-04-08 11 331 val_331 +2008-04-08 11 401 val_401 +2008-04-08 11 242 val_242 +2008-04-08 11 452 val_452 +2008-04-08 11 177 val_177 +2008-04-08 11 226 val_226 +2008-04-08 11 5 val_5 +2008-04-08 11 497 val_497 +2008-04-08 11 402 val_402 +2008-04-08 11 396 val_396 +2008-04-08 11 317 val_317 +2008-04-08 11 395 val_395 +2008-04-08 11 58 val_58 +2008-04-08 11 35 val_35 +2008-04-08 11 336 val_336 +2008-04-08 11 95 val_95 +2008-04-08 11 11 val_11 +2008-04-08 11 168 val_168 +2008-04-08 11 34 val_34 +2008-04-08 11 229 val_229 +2008-04-08 11 233 val_233 +2008-04-08 11 143 val_143 +2008-04-08 11 472 val_472 +2008-04-08 11 322 val_322 +2008-04-08 11 498 val_498 +2008-04-08 11 160 val_160 +2008-04-08 11 195 val_195 +2008-04-08 11 42 val_42 +2008-04-08 11 321 val_321 +2008-04-08 11 430 val_430 +2008-04-08 11 119 val_119 +2008-04-08 11 489 val_489 +2008-04-08 11 458 val_458 +2008-04-08 11 78 val_78 +2008-04-08 11 76 val_76 +2008-04-08 11 41 val_41 +2008-04-08 11 223 val_223 +2008-04-08 11 492 val_492 +2008-04-08 11 149 val_149 +2008-04-08 11 449 val_449 +2008-04-08 11 218 val_218 +2008-04-08 11 228 val_228 +2008-04-08 11 138 val_138 +2008-04-08 11 453 val_453 +2008-04-08 11 30 val_30 +2008-04-08 11 209 val_209 +2008-04-08 11 64 val_64 +2008-04-08 11 468 val_468 +2008-04-08 11 76 val_76 +2008-04-08 11 74 val_74 +2008-04-08 11 342 val_342 +2008-04-08 11 69 val_69 +2008-04-08 11 230 val_230 +2008-04-08 11 33 val_33 +2008-04-08 11 368 val_368 +2008-04-08 11 103 val_103 +2008-04-08 11 296 val_296 +2008-04-08 11 113 val_113 +2008-04-08 11 216 val_216 +2008-04-08 11 367 val_367 +2008-04-08 11 344 val_344 +2008-04-08 11 167 val_167 +2008-04-08 11 274 val_274 +2008-04-08 11 219 val_219 +2008-04-08 11 239 val_239 +2008-04-08 11 485 val_485 +2008-04-08 11 116 val_116 +2008-04-08 11 223 val_223 +2008-04-08 11 256 val_256 +2008-04-08 11 263 val_263 +2008-04-08 11 70 val_70 +2008-04-08 11 487 val_487 +2008-04-08 11 480 val_480 +2008-04-08 11 401 val_401 +2008-04-08 11 288 val_288 +2008-04-08 11 191 val_191 +2008-04-08 11 5 val_5 +2008-04-08 11 244 val_244 +2008-04-08 11 438 val_438 +2008-04-08 11 128 val_128 +2008-04-08 11 467 val_467 +2008-04-08 11 432 val_432 +2008-04-08 11 202 val_202 +2008-04-08 11 316 val_316 +2008-04-08 11 229 val_229 +2008-04-08 11 469 val_469 +2008-04-08 11 463 val_463 +2008-04-08 11 280 val_280 +2008-04-08 11 2 val_2 +2008-04-08 11 35 val_35 +2008-04-08 11 283 val_283 +2008-04-08 11 331 val_331 +2008-04-08 11 235 val_235 +2008-04-08 11 80 val_80 +2008-04-08 11 44 val_44 +2008-04-08 11 193 val_193 +2008-04-08 11 321 val_321 +2008-04-08 11 335 val_335 +2008-04-08 11 104 val_104 +2008-04-08 11 466 val_466 +2008-04-08 11 366 val_366 +2008-04-08 11 175 val_175 +2008-04-08 11 403 val_403 +2008-04-08 11 483 val_483 +2008-04-08 11 53 val_53 +2008-04-08 11 105 val_105 +2008-04-08 11 257 val_257 +2008-04-08 11 406 val_406 +2008-04-08 11 409 val_409 +2008-04-08 11 190 val_190 +2008-04-08 11 406 val_406 +2008-04-08 11 401 val_401 +2008-04-08 11 114 val_114 +2008-04-08 11 258 val_258 +2008-04-08 11 90 val_90 +2008-04-08 11 203 val_203 +2008-04-08 11 262 val_262 +2008-04-08 11 348 val_348 +2008-04-08 11 424 val_424 +2008-04-08 11 12 val_12 +2008-04-08 11 396 val_396 +2008-04-08 11 201 val_201 +2008-04-08 11 217 val_217 +2008-04-08 11 164 val_164 +2008-04-08 11 431 val_431 +2008-04-08 11 454 val_454 +2008-04-08 11 478 val_478 +2008-04-08 11 298 val_298 +2008-04-08 11 125 val_125 +2008-04-08 11 431 val_431 +2008-04-08 11 164 val_164 +2008-04-08 11 424 val_424 +2008-04-08 11 187 val_187 +2008-04-08 11 382 val_382 +2008-04-08 11 5 val_5 +2008-04-08 11 70 val_70 +2008-04-08 11 397 val_397 +2008-04-08 11 480 val_480 +2008-04-08 11 291 val_291 +2008-04-08 11 24 val_24 +2008-04-08 11 351 val_351 +2008-04-08 11 255 val_255 +2008-04-08 11 104 val_104 +2008-04-08 11 70 val_70 +2008-04-08 11 163 val_163 +2008-04-08 11 438 val_438 +2008-04-08 11 119 val_119 +2008-04-08 11 414 val_414 +2008-04-08 11 200 val_200 +2008-04-08 11 491 val_491 +2008-04-08 11 237 val_237 +2008-04-08 11 439 val_439 +2008-04-08 11 360 val_360 +2008-04-08 11 248 val_248 +2008-04-08 11 479 val_479 +2008-04-08 11 305 val_305 +2008-04-08 11 417 val_417 +2008-04-08 11 199 val_199 +2008-04-08 11 444 val_444 +2008-04-08 11 120 val_120 +2008-04-08 11 429 val_429 +2008-04-08 11 169 val_169 +2008-04-08 11 443 val_443 +2008-04-08 11 323 val_323 +2008-04-08 11 325 val_325 +2008-04-08 11 277 val_277 +2008-04-08 11 230 val_230 +2008-04-08 11 478 val_478 +2008-04-08 11 178 val_178 +2008-04-08 11 468 val_468 +2008-04-08 11 310 val_310 +2008-04-08 11 317 val_317 +2008-04-08 11 333 val_333 +2008-04-08 11 493 val_493 +2008-04-08 11 460 val_460 +2008-04-08 11 207 val_207 +2008-04-08 11 249 val_249 +2008-04-08 11 265 val_265 +2008-04-08 11 480 val_480 +2008-04-08 11 83 val_83 +2008-04-08 11 136 val_136 +2008-04-08 11 353 val_353 +2008-04-08 11 172 val_172 +2008-04-08 11 214 val_214 +2008-04-08 11 462 val_462 +2008-04-08 11 233 val_233 +2008-04-08 11 406 val_406 +2008-04-08 11 133 val_133 +2008-04-08 11 175 val_175 +2008-04-08 11 189 val_189 +2008-04-08 11 454 val_454 +2008-04-08 11 375 val_375 +2008-04-08 11 401 val_401 +2008-04-08 11 421 val_421 +2008-04-08 11 407 val_407 +2008-04-08 11 384 val_384 +2008-04-08 11 256 val_256 +2008-04-08 11 26 val_26 +2008-04-08 11 134 val_134 +2008-04-08 11 67 val_67 +2008-04-08 11 384 val_384 +2008-04-08 11 379 val_379 +2008-04-08 11 18 val_18 +2008-04-08 11 462 val_462 +2008-04-08 11 492 val_492 +2008-04-08 11 100 val_100 +2008-04-08 11 298 val_298 +2008-04-08 11 9 val_9 +2008-04-08 11 341 val_341 +2008-04-08 11 498 val_498 +2008-04-08 11 146 val_146 +2008-04-08 11 458 val_458 +2008-04-08 11 362 val_362 +2008-04-08 11 186 val_186 +2008-04-08 11 285 val_285 +2008-04-08 11 348 val_348 +2008-04-08 11 167 val_167 +2008-04-08 11 18 val_18 +2008-04-08 11 273 val_273 +2008-04-08 11 183 val_183 +2008-04-08 11 281 val_281 +2008-04-08 11 344 val_344 +2008-04-08 11 97 val_97 +2008-04-08 11 469 val_469 +2008-04-08 11 315 val_315 +2008-04-08 11 84 val_84 +2008-04-08 11 28 val_28 +2008-04-08 11 37 val_37 +2008-04-08 11 448 val_448 +2008-04-08 11 152 val_152 +2008-04-08 11 348 val_348 +2008-04-08 11 307 val_307 +2008-04-08 11 194 val_194 +2008-04-08 11 414 val_414 +2008-04-08 11 477 val_477 +2008-04-08 11 222 val_222 +2008-04-08 11 126 val_126 +2008-04-08 11 90 val_90 +2008-04-08 11 169 val_169 +2008-04-08 11 403 val_403 +2008-04-08 11 400 val_400 +2008-04-08 11 200 val_200 +2008-04-08 11 97 val_97 +2008-04-08 12 238 val_238 +2008-04-08 12 86 val_86 +2008-04-08 12 311 val_311 +2008-04-08 12 27 val_27 +2008-04-08 12 165 val_165 +2008-04-08 12 409 val_409 +2008-04-08 12 255 val_255 +2008-04-08 12 278 val_278 +2008-04-08 12 98 val_98 +2008-04-08 12 484 val_484 +2008-04-08 12 265 val_265 +2008-04-08 12 193 val_193 +2008-04-08 12 401 val_401 +2008-04-08 12 150 val_150 +2008-04-08 12 273 val_273 +2008-04-08 12 224 val_224 +2008-04-08 12 369 val_369 +2008-04-08 12 66 val_66 +2008-04-08 12 128 val_128 +2008-04-08 12 213 val_213 +2008-04-08 12 146 val_146 +2008-04-08 12 406 val_406 +2008-04-08 12 429 val_429 +2008-04-08 12 374 val_374 +2008-04-08 12 152 val_152 +2008-04-08 12 469 val_469 +2008-04-08 12 145 val_145 +2008-04-08 12 495 val_495 +2008-04-08 12 37 val_37 +2008-04-08 12 327 val_327 +2008-04-08 12 281 val_281 +2008-04-08 12 277 val_277 +2008-04-08 12 209 val_209 +2008-04-08 12 15 val_15 +2008-04-08 12 82 val_82 +2008-04-08 12 403 val_403 +2008-04-08 12 166 val_166 +2008-04-08 12 417 val_417 +2008-04-08 12 430 val_430 +2008-04-08 12 252 val_252 +2008-04-08 12 292 val_292 +2008-04-08 12 219 val_219 +2008-04-08 12 287 val_287 +2008-04-08 12 153 val_153 +2008-04-08 12 193 val_193 +2008-04-08 12 338 val_338 +2008-04-08 12 446 val_446 +2008-04-08 12 459 val_459 +2008-04-08 12 394 val_394 +2008-04-08 12 237 val_237 +2008-04-08 12 482 val_482 +2008-04-08 12 174 val_174 +2008-04-08 12 413 val_413 +2008-04-08 12 494 val_494 +2008-04-08 12 207 val_207 +2008-04-08 12 199 val_199 +2008-04-08 12 466 val_466 +2008-04-08 12 208 val_208 +2008-04-08 12 174 val_174 +2008-04-08 12 399 val_399 +2008-04-08 12 396 val_396 +2008-04-08 12 247 val_247 +2008-04-08 12 417 val_417 +2008-04-08 12 489 val_489 +2008-04-08 12 162 val_162 +2008-04-08 12 377 val_377 +2008-04-08 12 397 val_397 +2008-04-08 12 309 val_309 +2008-04-08 12 365 val_365 +2008-04-08 12 266 val_266 +2008-04-08 12 439 val_439 +2008-04-08 12 342 val_342 +2008-04-08 12 367 val_367 +2008-04-08 12 325 val_325 +2008-04-08 12 167 val_167 +2008-04-08 12 195 val_195 +2008-04-08 12 475 val_475 +2008-04-08 12 17 val_17 +2008-04-08 12 113 val_113 +2008-04-08 12 155 val_155 +2008-04-08 12 203 val_203 +2008-04-08 12 339 val_339 +2008-04-08 12 0 val_0 +2008-04-08 12 455 val_455 +2008-04-08 12 128 val_128 +2008-04-08 12 311 val_311 +2008-04-08 12 316 val_316 +2008-04-08 12 57 val_57 +2008-04-08 12 302 val_302 +2008-04-08 12 205 val_205 +2008-04-08 12 149 val_149 +2008-04-08 12 438 val_438 +2008-04-08 12 345 val_345 +2008-04-08 12 129 val_129 +2008-04-08 12 170 val_170 +2008-04-08 12 20 val_20 +2008-04-08 12 489 val_489 +2008-04-08 12 157 val_157 +2008-04-08 12 378 val_378 +2008-04-08 12 221 val_221 +2008-04-08 12 92 val_92 +2008-04-08 12 111 val_111 +2008-04-08 12 47 val_47 +2008-04-08 12 72 val_72 +2008-04-08 12 4 val_4 +2008-04-08 12 280 val_280 +2008-04-08 12 35 val_35 +2008-04-08 12 427 val_427 +2008-04-08 12 277 val_277 +2008-04-08 12 208 val_208 +2008-04-08 12 356 val_356 +2008-04-08 12 399 val_399 +2008-04-08 12 169 val_169 +2008-04-08 12 382 val_382 +2008-04-08 12 498 val_498 +2008-04-08 12 125 val_125 +2008-04-08 12 386 val_386 +2008-04-08 12 437 val_437 +2008-04-08 12 469 val_469 +2008-04-08 12 192 val_192 +2008-04-08 12 286 val_286 +2008-04-08 12 187 val_187 +2008-04-08 12 176 val_176 +2008-04-08 12 54 val_54 +2008-04-08 12 459 val_459 +2008-04-08 12 51 val_51 +2008-04-08 12 138 val_138 +2008-04-08 12 103 val_103 +2008-04-08 12 239 val_239 +2008-04-08 12 213 val_213 +2008-04-08 12 216 val_216 +2008-04-08 12 430 val_430 +2008-04-08 12 278 val_278 +2008-04-08 12 176 val_176 +2008-04-08 12 289 val_289 +2008-04-08 12 221 val_221 +2008-04-08 12 65 val_65 +2008-04-08 12 318 val_318 +2008-04-08 12 332 val_332 +2008-04-08 12 311 val_311 +2008-04-08 12 275 val_275 +2008-04-08 12 137 val_137 +2008-04-08 12 241 val_241 +2008-04-08 12 83 val_83 +2008-04-08 12 333 val_333 +2008-04-08 12 180 val_180 +2008-04-08 12 284 val_284 +2008-04-08 12 12 val_12 +2008-04-08 12 230 val_230 +2008-04-08 12 181 val_181 +2008-04-08 12 67 val_67 +2008-04-08 12 260 val_260 +2008-04-08 12 404 val_404 +2008-04-08 12 384 val_384 +2008-04-08 12 489 val_489 +2008-04-08 12 353 val_353 +2008-04-08 12 373 val_373 +2008-04-08 12 272 val_272 +2008-04-08 12 138 val_138 +2008-04-08 12 217 val_217 +2008-04-08 12 84 val_84 +2008-04-08 12 348 val_348 +2008-04-08 12 466 val_466 +2008-04-08 12 58 val_58 +2008-04-08 12 8 val_8 +2008-04-08 12 411 val_411 +2008-04-08 12 230 val_230 +2008-04-08 12 208 val_208 +2008-04-08 12 348 val_348 +2008-04-08 12 24 val_24 +2008-04-08 12 463 val_463 +2008-04-08 12 431 val_431 +2008-04-08 12 179 val_179 +2008-04-08 12 172 val_172 +2008-04-08 12 42 val_42 +2008-04-08 12 129 val_129 +2008-04-08 12 158 val_158 +2008-04-08 12 119 val_119 +2008-04-08 12 496 val_496 +2008-04-08 12 0 val_0 +2008-04-08 12 322 val_322 +2008-04-08 12 197 val_197 +2008-04-08 12 468 val_468 +2008-04-08 12 393 val_393 +2008-04-08 12 454 val_454 +2008-04-08 12 100 val_100 +2008-04-08 12 298 val_298 +2008-04-08 12 199 val_199 +2008-04-08 12 191 val_191 +2008-04-08 12 418 val_418 +2008-04-08 12 96 val_96 +2008-04-08 12 26 val_26 +2008-04-08 12 165 val_165 +2008-04-08 12 327 val_327 +2008-04-08 12 230 val_230 +2008-04-08 12 205 val_205 +2008-04-08 12 120 val_120 +2008-04-08 12 131 val_131 +2008-04-08 12 51 val_51 +2008-04-08 12 404 val_404 +2008-04-08 12 43 val_43 +2008-04-08 12 436 val_436 +2008-04-08 12 156 val_156 +2008-04-08 12 469 val_469 +2008-04-08 12 468 val_468 +2008-04-08 12 308 val_308 +2008-04-08 12 95 val_95 +2008-04-08 12 196 val_196 +2008-04-08 12 288 val_288 +2008-04-08 12 481 val_481 +2008-04-08 12 457 val_457 +2008-04-08 12 98 val_98 +2008-04-08 12 282 val_282 +2008-04-08 12 197 val_197 +2008-04-08 12 187 val_187 +2008-04-08 12 318 val_318 +2008-04-08 12 318 val_318 +2008-04-08 12 409 val_409 +2008-04-08 12 470 val_470 +2008-04-08 12 137 val_137 +2008-04-08 12 369 val_369 +2008-04-08 12 316 val_316 +2008-04-08 12 169 val_169 +2008-04-08 12 413 val_413 +2008-04-08 12 85 val_85 +2008-04-08 12 77 val_77 +2008-04-08 12 0 val_0 +2008-04-08 12 490 val_490 +2008-04-08 12 87 val_87 +2008-04-08 12 364 val_364 +2008-04-08 12 179 val_179 +2008-04-08 12 118 val_118 +2008-04-08 12 134 val_134 +2008-04-08 12 395 val_395 +2008-04-08 12 282 val_282 +2008-04-08 12 138 val_138 +2008-04-08 12 238 val_238 +2008-04-08 12 419 val_419 +2008-04-08 12 15 val_15 +2008-04-08 12 118 val_118 +2008-04-08 12 72 val_72 +2008-04-08 12 90 val_90 +2008-04-08 12 307 val_307 +2008-04-08 12 19 val_19 +2008-04-08 12 435 val_435 +2008-04-08 12 10 val_10 +2008-04-08 12 277 val_277 +2008-04-08 12 273 val_273 +2008-04-08 12 306 val_306 +2008-04-08 12 224 val_224 +2008-04-08 12 309 val_309 +2008-04-08 12 389 val_389 +2008-04-08 12 327 val_327 +2008-04-08 12 242 val_242 +2008-04-08 12 369 val_369 +2008-04-08 12 392 val_392 +2008-04-08 12 272 val_272 +2008-04-08 12 331 val_331 +2008-04-08 12 401 val_401 +2008-04-08 12 242 val_242 +2008-04-08 12 452 val_452 +2008-04-08 12 177 val_177 +2008-04-08 12 226 val_226 +2008-04-08 12 5 val_5 +2008-04-08 12 497 val_497 +2008-04-08 12 402 val_402 +2008-04-08 12 396 val_396 +2008-04-08 12 317 val_317 +2008-04-08 12 395 val_395 +2008-04-08 12 58 val_58 +2008-04-08 12 35 val_35 +2008-04-08 12 336 val_336 +2008-04-08 12 95 val_95 +2008-04-08 12 11 val_11 +2008-04-08 12 168 val_168 +2008-04-08 12 34 val_34 +2008-04-08 12 229 val_229 +2008-04-08 12 233 val_233 +2008-04-08 12 143 val_143 +2008-04-08 12 472 val_472 +2008-04-08 12 322 val_322 +2008-04-08 12 498 val_498 +2008-04-08 12 160 val_160 +2008-04-08 12 195 val_195 +2008-04-08 12 42 val_42 +2008-04-08 12 321 val_321 +2008-04-08 12 430 val_430 +2008-04-08 12 119 val_119 +2008-04-08 12 489 val_489 +2008-04-08 12 458 val_458 +2008-04-08 12 78 val_78 +2008-04-08 12 76 val_76 +2008-04-08 12 41 val_41 +2008-04-08 12 223 val_223 +2008-04-08 12 492 val_492 +2008-04-08 12 149 val_149 +2008-04-08 12 449 val_449 +2008-04-08 12 218 val_218 +2008-04-08 12 228 val_228 +2008-04-08 12 138 val_138 +2008-04-08 12 453 val_453 +2008-04-08 12 30 val_30 +2008-04-08 12 209 val_209 +2008-04-08 12 64 val_64 +2008-04-08 12 468 val_468 +2008-04-08 12 76 val_76 +2008-04-08 12 74 val_74 +2008-04-08 12 342 val_342 +2008-04-08 12 69 val_69 +2008-04-08 12 230 val_230 +2008-04-08 12 33 val_33 +2008-04-08 12 368 val_368 +2008-04-08 12 103 val_103 +2008-04-08 12 296 val_296 +2008-04-08 12 113 val_113 +2008-04-08 12 216 val_216 +2008-04-08 12 367 val_367 +2008-04-08 12 344 val_344 +2008-04-08 12 167 val_167 +2008-04-08 12 274 val_274 +2008-04-08 12 219 val_219 +2008-04-08 12 239 val_239 +2008-04-08 12 485 val_485 +2008-04-08 12 116 val_116 +2008-04-08 12 223 val_223 +2008-04-08 12 256 val_256 +2008-04-08 12 263 val_263 +2008-04-08 12 70 val_70 +2008-04-08 12 487 val_487 +2008-04-08 12 480 val_480 +2008-04-08 12 401 val_401 +2008-04-08 12 288 val_288 +2008-04-08 12 191 val_191 +2008-04-08 12 5 val_5 +2008-04-08 12 244 val_244 +2008-04-08 12 438 val_438 +2008-04-08 12 128 val_128 +2008-04-08 12 467 val_467 +2008-04-08 12 432 val_432 +2008-04-08 12 202 val_202 +2008-04-08 12 316 val_316 +2008-04-08 12 229 val_229 +2008-04-08 12 469 val_469 +2008-04-08 12 463 val_463 +2008-04-08 12 280 val_280 +2008-04-08 12 2 val_2 +2008-04-08 12 35 val_35 +2008-04-08 12 283 val_283 +2008-04-08 12 331 val_331 +2008-04-08 12 235 val_235 +2008-04-08 12 80 val_80 +2008-04-08 12 44 val_44 +2008-04-08 12 193 val_193 +2008-04-08 12 321 val_321 +2008-04-08 12 335 val_335 +2008-04-08 12 104 val_104 +2008-04-08 12 466 val_466 +2008-04-08 12 366 val_366 +2008-04-08 12 175 val_175 +2008-04-08 12 403 val_403 +2008-04-08 12 483 val_483 +2008-04-08 12 53 val_53 +2008-04-08 12 105 val_105 +2008-04-08 12 257 val_257 +2008-04-08 12 406 val_406 +2008-04-08 12 409 val_409 +2008-04-08 12 190 val_190 +2008-04-08 12 406 val_406 +2008-04-08 12 401 val_401 +2008-04-08 12 114 val_114 +2008-04-08 12 258 val_258 +2008-04-08 12 90 val_90 +2008-04-08 12 203 val_203 +2008-04-08 12 262 val_262 +2008-04-08 12 348 val_348 +2008-04-08 12 424 val_424 +2008-04-08 12 12 val_12 +2008-04-08 12 396 val_396 +2008-04-08 12 201 val_201 +2008-04-08 12 217 val_217 +2008-04-08 12 164 val_164 +2008-04-08 12 431 val_431 +2008-04-08 12 454 val_454 +2008-04-08 12 478 val_478 +2008-04-08 12 298 val_298 +2008-04-08 12 125 val_125 +2008-04-08 12 431 val_431 +2008-04-08 12 164 val_164 +2008-04-08 12 424 val_424 +2008-04-08 12 187 val_187 +2008-04-08 12 382 val_382 +2008-04-08 12 5 val_5 +2008-04-08 12 70 val_70 +2008-04-08 12 397 val_397 +2008-04-08 12 480 val_480 +2008-04-08 12 291 val_291 +2008-04-08 12 24 val_24 +2008-04-08 12 351 val_351 +2008-04-08 12 255 val_255 +2008-04-08 12 104 val_104 +2008-04-08 12 70 val_70 +2008-04-08 12 163 val_163 +2008-04-08 12 438 val_438 +2008-04-08 12 119 val_119 +2008-04-08 12 414 val_414 +2008-04-08 12 200 val_200 +2008-04-08 12 491 val_491 +2008-04-08 12 237 val_237 +2008-04-08 12 439 val_439 +2008-04-08 12 360 val_360 +2008-04-08 12 248 val_248 +2008-04-08 12 479 val_479 +2008-04-08 12 305 val_305 +2008-04-08 12 417 val_417 +2008-04-08 12 199 val_199 +2008-04-08 12 444 val_444 +2008-04-08 12 120 val_120 +2008-04-08 12 429 val_429 +2008-04-08 12 169 val_169 +2008-04-08 12 443 val_443 +2008-04-08 12 323 val_323 +2008-04-08 12 325 val_325 +2008-04-08 12 277 val_277 +2008-04-08 12 230 val_230 +2008-04-08 12 478 val_478 +2008-04-08 12 178 val_178 +2008-04-08 12 468 val_468 +2008-04-08 12 310 val_310 +2008-04-08 12 317 val_317 +2008-04-08 12 333 val_333 +2008-04-08 12 493 val_493 +2008-04-08 12 460 val_460 +2008-04-08 12 207 val_207 +2008-04-08 12 249 val_249 +2008-04-08 12 265 val_265 +2008-04-08 12 480 val_480 +2008-04-08 12 83 val_83 +2008-04-08 12 136 val_136 +2008-04-08 12 353 val_353 +2008-04-08 12 172 val_172 +2008-04-08 12 214 val_214 +2008-04-08 12 462 val_462 +2008-04-08 12 233 val_233 +2008-04-08 12 406 val_406 +2008-04-08 12 133 val_133 +2008-04-08 12 175 val_175 +2008-04-08 12 189 val_189 +2008-04-08 12 454 val_454 +2008-04-08 12 375 val_375 +2008-04-08 12 401 val_401 +2008-04-08 12 421 val_421 +2008-04-08 12 407 val_407 +2008-04-08 12 384 val_384 +2008-04-08 12 256 val_256 +2008-04-08 12 26 val_26 +2008-04-08 12 134 val_134 +2008-04-08 12 67 val_67 +2008-04-08 12 384 val_384 +2008-04-08 12 379 val_379 +2008-04-08 12 18 val_18 +2008-04-08 12 462 val_462 +2008-04-08 12 492 val_492 +2008-04-08 12 100 val_100 +2008-04-08 12 298 val_298 +2008-04-08 12 9 val_9 +2008-04-08 12 341 val_341 +2008-04-08 12 498 val_498 +2008-04-08 12 146 val_146 +2008-04-08 12 458 val_458 +2008-04-08 12 362 val_362 +2008-04-08 12 186 val_186 +2008-04-08 12 285 val_285 +2008-04-08 12 348 val_348 +2008-04-08 12 167 val_167 +2008-04-08 12 18 val_18 +2008-04-08 12 273 val_273 +2008-04-08 12 183 val_183 +2008-04-08 12 281 val_281 +2008-04-08 12 344 val_344 +2008-04-08 12 97 val_97 +2008-04-08 12 469 val_469 +2008-04-08 12 315 val_315 +2008-04-08 12 84 val_84 +2008-04-08 12 28 val_28 +2008-04-08 12 37 val_37 +2008-04-08 12 448 val_448 +2008-04-08 12 152 val_152 +2008-04-08 12 348 val_348 +2008-04-08 12 307 val_307 +2008-04-08 12 194 val_194 +2008-04-08 12 414 val_414 +2008-04-08 12 477 val_477 +2008-04-08 12 222 val_222 +2008-04-08 12 126 val_126 +2008-04-08 12 90 val_90 +2008-04-08 12 169 val_169 +2008-04-08 12 403 val_403 +2008-04-08 12 400 val_400 +2008-04-08 12 200 val_200 +2008-04-08 12 97 val_97 +2008-04-09 11 238 val_238 +2008-04-09 11 86 val_86 +2008-04-09 11 311 val_311 +2008-04-09 11 27 val_27 +2008-04-09 11 165 val_165 +2008-04-09 11 409 val_409 +2008-04-09 11 255 val_255 +2008-04-09 11 278 val_278 +2008-04-09 11 98 val_98 +2008-04-09 11 484 val_484 +2008-04-09 11 265 val_265 +2008-04-09 11 193 val_193 +2008-04-09 11 401 val_401 +2008-04-09 11 150 val_150 +2008-04-09 11 273 val_273 +2008-04-09 11 224 val_224 +2008-04-09 11 369 val_369 +2008-04-09 11 66 val_66 +2008-04-09 11 128 val_128 +2008-04-09 11 213 val_213 +2008-04-09 11 146 val_146 +2008-04-09 11 406 val_406 +2008-04-09 11 429 val_429 +2008-04-09 11 374 val_374 +2008-04-09 11 152 val_152 +2008-04-09 11 469 val_469 +2008-04-09 11 145 val_145 +2008-04-09 11 495 val_495 +2008-04-09 11 37 val_37 +2008-04-09 11 327 val_327 +2008-04-09 11 281 val_281 +2008-04-09 11 277 val_277 +2008-04-09 11 209 val_209 +2008-04-09 11 15 val_15 +2008-04-09 11 82 val_82 +2008-04-09 11 403 val_403 +2008-04-09 11 166 val_166 +2008-04-09 11 417 val_417 +2008-04-09 11 430 val_430 +2008-04-09 11 252 val_252 +2008-04-09 11 292 val_292 +2008-04-09 11 219 val_219 +2008-04-09 11 287 val_287 +2008-04-09 11 153 val_153 +2008-04-09 11 193 val_193 +2008-04-09 11 338 val_338 +2008-04-09 11 446 val_446 +2008-04-09 11 459 val_459 +2008-04-09 11 394 val_394 +2008-04-09 11 237 val_237 +2008-04-09 11 482 val_482 +2008-04-09 11 174 val_174 +2008-04-09 11 413 val_413 +2008-04-09 11 494 val_494 +2008-04-09 11 207 val_207 +2008-04-09 11 199 val_199 +2008-04-09 11 466 val_466 +2008-04-09 11 208 val_208 +2008-04-09 11 174 val_174 +2008-04-09 11 399 val_399 +2008-04-09 11 396 val_396 +2008-04-09 11 247 val_247 +2008-04-09 11 417 val_417 +2008-04-09 11 489 val_489 +2008-04-09 11 162 val_162 +2008-04-09 11 377 val_377 +2008-04-09 11 397 val_397 +2008-04-09 11 309 val_309 +2008-04-09 11 365 val_365 +2008-04-09 11 266 val_266 +2008-04-09 11 439 val_439 +2008-04-09 11 342 val_342 +2008-04-09 11 367 val_367 +2008-04-09 11 325 val_325 +2008-04-09 11 167 val_167 +2008-04-09 11 195 val_195 +2008-04-09 11 475 val_475 +2008-04-09 11 17 val_17 +2008-04-09 11 113 val_113 +2008-04-09 11 155 val_155 +2008-04-09 11 203 val_203 +2008-04-09 11 339 val_339 +2008-04-09 11 0 val_0 +2008-04-09 11 455 val_455 +2008-04-09 11 128 val_128 +2008-04-09 11 311 val_311 +2008-04-09 11 316 val_316 +2008-04-09 11 57 val_57 +2008-04-09 11 302 val_302 +2008-04-09 11 205 val_205 +2008-04-09 11 149 val_149 +2008-04-09 11 438 val_438 +2008-04-09 11 345 val_345 +2008-04-09 11 129 val_129 +2008-04-09 11 170 val_170 +2008-04-09 11 20 val_20 +2008-04-09 11 489 val_489 +2008-04-09 11 157 val_157 +2008-04-09 11 378 val_378 +2008-04-09 11 221 val_221 +2008-04-09 11 92 val_92 +2008-04-09 11 111 val_111 +2008-04-09 11 47 val_47 +2008-04-09 11 72 val_72 +2008-04-09 11 4 val_4 +2008-04-09 11 280 val_280 +2008-04-09 11 35 val_35 +2008-04-09 11 427 val_427 +2008-04-09 11 277 val_277 +2008-04-09 11 208 val_208 +2008-04-09 11 356 val_356 +2008-04-09 11 399 val_399 +2008-04-09 11 169 val_169 +2008-04-09 11 382 val_382 +2008-04-09 11 498 val_498 +2008-04-09 11 125 val_125 +2008-04-09 11 386 val_386 +2008-04-09 11 437 val_437 +2008-04-09 11 469 val_469 +2008-04-09 11 192 val_192 +2008-04-09 11 286 val_286 +2008-04-09 11 187 val_187 +2008-04-09 11 176 val_176 +2008-04-09 11 54 val_54 +2008-04-09 11 459 val_459 +2008-04-09 11 51 val_51 +2008-04-09 11 138 val_138 +2008-04-09 11 103 val_103 +2008-04-09 11 239 val_239 +2008-04-09 11 213 val_213 +2008-04-09 11 216 val_216 +2008-04-09 11 430 val_430 +2008-04-09 11 278 val_278 +2008-04-09 11 176 val_176 +2008-04-09 11 289 val_289 +2008-04-09 11 221 val_221 +2008-04-09 11 65 val_65 +2008-04-09 11 318 val_318 +2008-04-09 11 332 val_332 +2008-04-09 11 311 val_311 +2008-04-09 11 275 val_275 +2008-04-09 11 137 val_137 +2008-04-09 11 241 val_241 +2008-04-09 11 83 val_83 +2008-04-09 11 333 val_333 +2008-04-09 11 180 val_180 +2008-04-09 11 284 val_284 +2008-04-09 11 12 val_12 +2008-04-09 11 230 val_230 +2008-04-09 11 181 val_181 +2008-04-09 11 67 val_67 +2008-04-09 11 260 val_260 +2008-04-09 11 404 val_404 +2008-04-09 11 384 val_384 +2008-04-09 11 489 val_489 +2008-04-09 11 353 val_353 +2008-04-09 11 373 val_373 +2008-04-09 11 272 val_272 +2008-04-09 11 138 val_138 +2008-04-09 11 217 val_217 +2008-04-09 11 84 val_84 +2008-04-09 11 348 val_348 +2008-04-09 11 466 val_466 +2008-04-09 11 58 val_58 +2008-04-09 11 8 val_8 +2008-04-09 11 411 val_411 +2008-04-09 11 230 val_230 +2008-04-09 11 208 val_208 +2008-04-09 11 348 val_348 +2008-04-09 11 24 val_24 +2008-04-09 11 463 val_463 +2008-04-09 11 431 val_431 +2008-04-09 11 179 val_179 +2008-04-09 11 172 val_172 +2008-04-09 11 42 val_42 +2008-04-09 11 129 val_129 +2008-04-09 11 158 val_158 +2008-04-09 11 119 val_119 +2008-04-09 11 496 val_496 +2008-04-09 11 0 val_0 +2008-04-09 11 322 val_322 +2008-04-09 11 197 val_197 +2008-04-09 11 468 val_468 +2008-04-09 11 393 val_393 +2008-04-09 11 454 val_454 +2008-04-09 11 100 val_100 +2008-04-09 11 298 val_298 +2008-04-09 11 199 val_199 +2008-04-09 11 191 val_191 +2008-04-09 11 418 val_418 +2008-04-09 11 96 val_96 +2008-04-09 11 26 val_26 +2008-04-09 11 165 val_165 +2008-04-09 11 327 val_327 +2008-04-09 11 230 val_230 +2008-04-09 11 205 val_205 +2008-04-09 11 120 val_120 +2008-04-09 11 131 val_131 +2008-04-09 11 51 val_51 +2008-04-09 11 404 val_404 +2008-04-09 11 43 val_43 +2008-04-09 11 436 val_436 +2008-04-09 11 156 val_156 +2008-04-09 11 469 val_469 +2008-04-09 11 468 val_468 +2008-04-09 11 308 val_308 +2008-04-09 11 95 val_95 +2008-04-09 11 196 val_196 +2008-04-09 11 288 val_288 +2008-04-09 11 481 val_481 +2008-04-09 11 457 val_457 +2008-04-09 11 98 val_98 +2008-04-09 11 282 val_282 +2008-04-09 11 197 val_197 +2008-04-09 11 187 val_187 +2008-04-09 11 318 val_318 +2008-04-09 11 318 val_318 +2008-04-09 11 409 val_409 +2008-04-09 11 470 val_470 +2008-04-09 11 137 val_137 +2008-04-09 11 369 val_369 +2008-04-09 11 316 val_316 +2008-04-09 11 169 val_169 +2008-04-09 11 413 val_413 +2008-04-09 11 85 val_85 +2008-04-09 11 77 val_77 +2008-04-09 11 0 val_0 +2008-04-09 11 490 val_490 +2008-04-09 11 87 val_87 +2008-04-09 11 364 val_364 +2008-04-09 11 179 val_179 +2008-04-09 11 118 val_118 +2008-04-09 11 134 val_134 +2008-04-09 11 395 val_395 +2008-04-09 11 282 val_282 +2008-04-09 11 138 val_138 +2008-04-09 11 238 val_238 +2008-04-09 11 419 val_419 +2008-04-09 11 15 val_15 +2008-04-09 11 118 val_118 +2008-04-09 11 72 val_72 +2008-04-09 11 90 val_90 +2008-04-09 11 307 val_307 +2008-04-09 11 19 val_19 +2008-04-09 11 435 val_435 +2008-04-09 11 10 val_10 +2008-04-09 11 277 val_277 +2008-04-09 11 273 val_273 +2008-04-09 11 306 val_306 +2008-04-09 11 224 val_224 +2008-04-09 11 309 val_309 +2008-04-09 11 389 val_389 +2008-04-09 11 327 val_327 +2008-04-09 11 242 val_242 +2008-04-09 11 369 val_369 +2008-04-09 11 392 val_392 +2008-04-09 11 272 val_272 +2008-04-09 11 331 val_331 +2008-04-09 11 401 val_401 +2008-04-09 11 242 val_242 +2008-04-09 11 452 val_452 +2008-04-09 11 177 val_177 +2008-04-09 11 226 val_226 +2008-04-09 11 5 val_5 +2008-04-09 11 497 val_497 +2008-04-09 11 402 val_402 +2008-04-09 11 396 val_396 +2008-04-09 11 317 val_317 +2008-04-09 11 395 val_395 +2008-04-09 11 58 val_58 +2008-04-09 11 35 val_35 +2008-04-09 11 336 val_336 +2008-04-09 11 95 val_95 +2008-04-09 11 11 val_11 +2008-04-09 11 168 val_168 +2008-04-09 11 34 val_34 +2008-04-09 11 229 val_229 +2008-04-09 11 233 val_233 +2008-04-09 11 143 val_143 +2008-04-09 11 472 val_472 +2008-04-09 11 322 val_322 +2008-04-09 11 498 val_498 +2008-04-09 11 160 val_160 +2008-04-09 11 195 val_195 +2008-04-09 11 42 val_42 +2008-04-09 11 321 val_321 +2008-04-09 11 430 val_430 +2008-04-09 11 119 val_119 +2008-04-09 11 489 val_489 +2008-04-09 11 458 val_458 +2008-04-09 11 78 val_78 +2008-04-09 11 76 val_76 +2008-04-09 11 41 val_41 +2008-04-09 11 223 val_223 +2008-04-09 11 492 val_492 +2008-04-09 11 149 val_149 +2008-04-09 11 449 val_449 +2008-04-09 11 218 val_218 +2008-04-09 11 228 val_228 +2008-04-09 11 138 val_138 +2008-04-09 11 453 val_453 +2008-04-09 11 30 val_30 +2008-04-09 11 209 val_209 +2008-04-09 11 64 val_64 +2008-04-09 11 468 val_468 +2008-04-09 11 76 val_76 +2008-04-09 11 74 val_74 +2008-04-09 11 342 val_342 +2008-04-09 11 69 val_69 +2008-04-09 11 230 val_230 +2008-04-09 11 33 val_33 +2008-04-09 11 368 val_368 +2008-04-09 11 103 val_103 +2008-04-09 11 296 val_296 +2008-04-09 11 113 val_113 +2008-04-09 11 216 val_216 +2008-04-09 11 367 val_367 +2008-04-09 11 344 val_344 +2008-04-09 11 167 val_167 +2008-04-09 11 274 val_274 +2008-04-09 11 219 val_219 +2008-04-09 11 239 val_239 +2008-04-09 11 485 val_485 +2008-04-09 11 116 val_116 +2008-04-09 11 223 val_223 +2008-04-09 11 256 val_256 +2008-04-09 11 263 val_263 +2008-04-09 11 70 val_70 +2008-04-09 11 487 val_487 +2008-04-09 11 480 val_480 +2008-04-09 11 401 val_401 +2008-04-09 11 288 val_288 +2008-04-09 11 191 val_191 +2008-04-09 11 5 val_5 +2008-04-09 11 244 val_244 +2008-04-09 11 438 val_438 +2008-04-09 11 128 val_128 +2008-04-09 11 467 val_467 +2008-04-09 11 432 val_432 +2008-04-09 11 202 val_202 +2008-04-09 11 316 val_316 +2008-04-09 11 229 val_229 +2008-04-09 11 469 val_469 +2008-04-09 11 463 val_463 +2008-04-09 11 280 val_280 +2008-04-09 11 2 val_2 +2008-04-09 11 35 val_35 +2008-04-09 11 283 val_283 +2008-04-09 11 331 val_331 +2008-04-09 11 235 val_235 +2008-04-09 11 80 val_80 +2008-04-09 11 44 val_44 +2008-04-09 11 193 val_193 +2008-04-09 11 321 val_321 +2008-04-09 11 335 val_335 +2008-04-09 11 104 val_104 +2008-04-09 11 466 val_466 +2008-04-09 11 366 val_366 +2008-04-09 11 175 val_175 +2008-04-09 11 403 val_403 +2008-04-09 11 483 val_483 +2008-04-09 11 53 val_53 +2008-04-09 11 105 val_105 +2008-04-09 11 257 val_257 +2008-04-09 11 406 val_406 +2008-04-09 11 409 val_409 +2008-04-09 11 190 val_190 +2008-04-09 11 406 val_406 +2008-04-09 11 401 val_401 +2008-04-09 11 114 val_114 +2008-04-09 11 258 val_258 +2008-04-09 11 90 val_90 +2008-04-09 11 203 val_203 +2008-04-09 11 262 val_262 +2008-04-09 11 348 val_348 +2008-04-09 11 424 val_424 +2008-04-09 11 12 val_12 +2008-04-09 11 396 val_396 +2008-04-09 11 201 val_201 +2008-04-09 11 217 val_217 +2008-04-09 11 164 val_164 +2008-04-09 11 431 val_431 +2008-04-09 11 454 val_454 +2008-04-09 11 478 val_478 +2008-04-09 11 298 val_298 +2008-04-09 11 125 val_125 +2008-04-09 11 431 val_431 +2008-04-09 11 164 val_164 +2008-04-09 11 424 val_424 +2008-04-09 11 187 val_187 +2008-04-09 11 382 val_382 +2008-04-09 11 5 val_5 +2008-04-09 11 70 val_70 +2008-04-09 11 397 val_397 +2008-04-09 11 480 val_480 +2008-04-09 11 291 val_291 +2008-04-09 11 24 val_24 +2008-04-09 11 351 val_351 +2008-04-09 11 255 val_255 +2008-04-09 11 104 val_104 +2008-04-09 11 70 val_70 +2008-04-09 11 163 val_163 +2008-04-09 11 438 val_438 +2008-04-09 11 119 val_119 +2008-04-09 11 414 val_414 +2008-04-09 11 200 val_200 +2008-04-09 11 491 val_491 +2008-04-09 11 237 val_237 +2008-04-09 11 439 val_439 +2008-04-09 11 360 val_360 +2008-04-09 11 248 val_248 +2008-04-09 11 479 val_479 +2008-04-09 11 305 val_305 +2008-04-09 11 417 val_417 +2008-04-09 11 199 val_199 +2008-04-09 11 444 val_444 +2008-04-09 11 120 val_120 +2008-04-09 11 429 val_429 +2008-04-09 11 169 val_169 +2008-04-09 11 443 val_443 +2008-04-09 11 323 val_323 +2008-04-09 11 325 val_325 +2008-04-09 11 277 val_277 +2008-04-09 11 230 val_230 +2008-04-09 11 478 val_478 +2008-04-09 11 178 val_178 +2008-04-09 11 468 val_468 +2008-04-09 11 310 val_310 +2008-04-09 11 317 val_317 +2008-04-09 11 333 val_333 +2008-04-09 11 493 val_493 +2008-04-09 11 460 val_460 +2008-04-09 11 207 val_207 +2008-04-09 11 249 val_249 +2008-04-09 11 265 val_265 +2008-04-09 11 480 val_480 +2008-04-09 11 83 val_83 +2008-04-09 11 136 val_136 +2008-04-09 11 353 val_353 +2008-04-09 11 172 val_172 +2008-04-09 11 214 val_214 +2008-04-09 11 462 val_462 +2008-04-09 11 233 val_233 +2008-04-09 11 406 val_406 +2008-04-09 11 133 val_133 +2008-04-09 11 175 val_175 +2008-04-09 11 189 val_189 +2008-04-09 11 454 val_454 +2008-04-09 11 375 val_375 +2008-04-09 11 401 val_401 +2008-04-09 11 421 val_421 +2008-04-09 11 407 val_407 +2008-04-09 11 384 val_384 +2008-04-09 11 256 val_256 +2008-04-09 11 26 val_26 +2008-04-09 11 134 val_134 +2008-04-09 11 67 val_67 +2008-04-09 11 384 val_384 +2008-04-09 11 379 val_379 +2008-04-09 11 18 val_18 +2008-04-09 11 462 val_462 +2008-04-09 11 492 val_492 +2008-04-09 11 100 val_100 +2008-04-09 11 298 val_298 +2008-04-09 11 9 val_9 +2008-04-09 11 341 val_341 +2008-04-09 11 498 val_498 +2008-04-09 11 146 val_146 +2008-04-09 11 458 val_458 +2008-04-09 11 362 val_362 +2008-04-09 11 186 val_186 +2008-04-09 11 285 val_285 +2008-04-09 11 348 val_348 +2008-04-09 11 167 val_167 +2008-04-09 11 18 val_18 +2008-04-09 11 273 val_273 +2008-04-09 11 183 val_183 +2008-04-09 11 281 val_281 +2008-04-09 11 344 val_344 +2008-04-09 11 97 val_97 +2008-04-09 11 469 val_469 +2008-04-09 11 315 val_315 +2008-04-09 11 84 val_84 +2008-04-09 11 28 val_28 +2008-04-09 11 37 val_37 +2008-04-09 11 448 val_448 +2008-04-09 11 152 val_152 +2008-04-09 11 348 val_348 +2008-04-09 11 307 val_307 +2008-04-09 11 194 val_194 +2008-04-09 11 414 val_414 +2008-04-09 11 477 val_477 +2008-04-09 11 222 val_222 +2008-04-09 11 126 val_126 +2008-04-09 11 90 val_90 +2008-04-09 11 169 val_169 +2008-04-09 11 403 val_403 +2008-04-09 11 400 val_400 +2008-04-09 11 200 val_200 +2008-04-09 11 97 val_97 +2008-04-09 12 238 val_238 +2008-04-09 12 86 val_86 +2008-04-09 12 311 val_311 +2008-04-09 12 27 val_27 +2008-04-09 12 165 val_165 +2008-04-09 12 409 val_409 +2008-04-09 12 255 val_255 +2008-04-09 12 278 val_278 +2008-04-09 12 98 val_98 +2008-04-09 12 484 val_484 +2008-04-09 12 265 val_265 +2008-04-09 12 193 val_193 +2008-04-09 12 401 val_401 +2008-04-09 12 150 val_150 +2008-04-09 12 273 val_273 +2008-04-09 12 224 val_224 +2008-04-09 12 369 val_369 +2008-04-09 12 66 val_66 +2008-04-09 12 128 val_128 +2008-04-09 12 213 val_213 +2008-04-09 12 146 val_146 +2008-04-09 12 406 val_406 +2008-04-09 12 429 val_429 +2008-04-09 12 374 val_374 +2008-04-09 12 152 val_152 +2008-04-09 12 469 val_469 +2008-04-09 12 145 val_145 +2008-04-09 12 495 val_495 +2008-04-09 12 37 val_37 +2008-04-09 12 327 val_327 +2008-04-09 12 281 val_281 +2008-04-09 12 277 val_277 +2008-04-09 12 209 val_209 +2008-04-09 12 15 val_15 +2008-04-09 12 82 val_82 +2008-04-09 12 403 val_403 +2008-04-09 12 166 val_166 +2008-04-09 12 417 val_417 +2008-04-09 12 430 val_430 +2008-04-09 12 252 val_252 +2008-04-09 12 292 val_292 +2008-04-09 12 219 val_219 +2008-04-09 12 287 val_287 +2008-04-09 12 153 val_153 +2008-04-09 12 193 val_193 +2008-04-09 12 338 val_338 +2008-04-09 12 446 val_446 +2008-04-09 12 459 val_459 +2008-04-09 12 394 val_394 +2008-04-09 12 237 val_237 +2008-04-09 12 482 val_482 +2008-04-09 12 174 val_174 +2008-04-09 12 413 val_413 +2008-04-09 12 494 val_494 +2008-04-09 12 207 val_207 +2008-04-09 12 199 val_199 +2008-04-09 12 466 val_466 +2008-04-09 12 208 val_208 +2008-04-09 12 174 val_174 +2008-04-09 12 399 val_399 +2008-04-09 12 396 val_396 +2008-04-09 12 247 val_247 +2008-04-09 12 417 val_417 +2008-04-09 12 489 val_489 +2008-04-09 12 162 val_162 +2008-04-09 12 377 val_377 +2008-04-09 12 397 val_397 +2008-04-09 12 309 val_309 +2008-04-09 12 365 val_365 +2008-04-09 12 266 val_266 +2008-04-09 12 439 val_439 +2008-04-09 12 342 val_342 +2008-04-09 12 367 val_367 +2008-04-09 12 325 val_325 +2008-04-09 12 167 val_167 +2008-04-09 12 195 val_195 +2008-04-09 12 475 val_475 +2008-04-09 12 17 val_17 +2008-04-09 12 113 val_113 +2008-04-09 12 155 val_155 +2008-04-09 12 203 val_203 +2008-04-09 12 339 val_339 +2008-04-09 12 0 val_0 +2008-04-09 12 455 val_455 +2008-04-09 12 128 val_128 +2008-04-09 12 311 val_311 +2008-04-09 12 316 val_316 +2008-04-09 12 57 val_57 +2008-04-09 12 302 val_302 +2008-04-09 12 205 val_205 +2008-04-09 12 149 val_149 +2008-04-09 12 438 val_438 +2008-04-09 12 345 val_345 +2008-04-09 12 129 val_129 +2008-04-09 12 170 val_170 +2008-04-09 12 20 val_20 +2008-04-09 12 489 val_489 +2008-04-09 12 157 val_157 +2008-04-09 12 378 val_378 +2008-04-09 12 221 val_221 +2008-04-09 12 92 val_92 +2008-04-09 12 111 val_111 +2008-04-09 12 47 val_47 +2008-04-09 12 72 val_72 +2008-04-09 12 4 val_4 +2008-04-09 12 280 val_280 +2008-04-09 12 35 val_35 +2008-04-09 12 427 val_427 +2008-04-09 12 277 val_277 +2008-04-09 12 208 val_208 +2008-04-09 12 356 val_356 +2008-04-09 12 399 val_399 +2008-04-09 12 169 val_169 +2008-04-09 12 382 val_382 +2008-04-09 12 498 val_498 +2008-04-09 12 125 val_125 +2008-04-09 12 386 val_386 +2008-04-09 12 437 val_437 +2008-04-09 12 469 val_469 +2008-04-09 12 192 val_192 +2008-04-09 12 286 val_286 +2008-04-09 12 187 val_187 +2008-04-09 12 176 val_176 +2008-04-09 12 54 val_54 +2008-04-09 12 459 val_459 +2008-04-09 12 51 val_51 +2008-04-09 12 138 val_138 +2008-04-09 12 103 val_103 +2008-04-09 12 239 val_239 +2008-04-09 12 213 val_213 +2008-04-09 12 216 val_216 +2008-04-09 12 430 val_430 +2008-04-09 12 278 val_278 +2008-04-09 12 176 val_176 +2008-04-09 12 289 val_289 +2008-04-09 12 221 val_221 +2008-04-09 12 65 val_65 +2008-04-09 12 318 val_318 +2008-04-09 12 332 val_332 +2008-04-09 12 311 val_311 +2008-04-09 12 275 val_275 +2008-04-09 12 137 val_137 +2008-04-09 12 241 val_241 +2008-04-09 12 83 val_83 +2008-04-09 12 333 val_333 +2008-04-09 12 180 val_180 +2008-04-09 12 284 val_284 +2008-04-09 12 12 val_12 +2008-04-09 12 230 val_230 +2008-04-09 12 181 val_181 +2008-04-09 12 67 val_67 +2008-04-09 12 260 val_260 +2008-04-09 12 404 val_404 +2008-04-09 12 384 val_384 +2008-04-09 12 489 val_489 +2008-04-09 12 353 val_353 +2008-04-09 12 373 val_373 +2008-04-09 12 272 val_272 +2008-04-09 12 138 val_138 +2008-04-09 12 217 val_217 +2008-04-09 12 84 val_84 +2008-04-09 12 348 val_348 +2008-04-09 12 466 val_466 +2008-04-09 12 58 val_58 +2008-04-09 12 8 val_8 +2008-04-09 12 411 val_411 +2008-04-09 12 230 val_230 +2008-04-09 12 208 val_208 +2008-04-09 12 348 val_348 +2008-04-09 12 24 val_24 +2008-04-09 12 463 val_463 +2008-04-09 12 431 val_431 +2008-04-09 12 179 val_179 +2008-04-09 12 172 val_172 +2008-04-09 12 42 val_42 +2008-04-09 12 129 val_129 +2008-04-09 12 158 val_158 +2008-04-09 12 119 val_119 +2008-04-09 12 496 val_496 +2008-04-09 12 0 val_0 +2008-04-09 12 322 val_322 +2008-04-09 12 197 val_197 +2008-04-09 12 468 val_468 +2008-04-09 12 393 val_393 +2008-04-09 12 454 val_454 +2008-04-09 12 100 val_100 +2008-04-09 12 298 val_298 +2008-04-09 12 199 val_199 +2008-04-09 12 191 val_191 +2008-04-09 12 418 val_418 +2008-04-09 12 96 val_96 +2008-04-09 12 26 val_26 +2008-04-09 12 165 val_165 +2008-04-09 12 327 val_327 +2008-04-09 12 230 val_230 +2008-04-09 12 205 val_205 +2008-04-09 12 120 val_120 +2008-04-09 12 131 val_131 +2008-04-09 12 51 val_51 +2008-04-09 12 404 val_404 +2008-04-09 12 43 val_43 +2008-04-09 12 436 val_436 +2008-04-09 12 156 val_156 +2008-04-09 12 469 val_469 +2008-04-09 12 468 val_468 +2008-04-09 12 308 val_308 +2008-04-09 12 95 val_95 +2008-04-09 12 196 val_196 +2008-04-09 12 288 val_288 +2008-04-09 12 481 val_481 +2008-04-09 12 457 val_457 +2008-04-09 12 98 val_98 +2008-04-09 12 282 val_282 +2008-04-09 12 197 val_197 +2008-04-09 12 187 val_187 +2008-04-09 12 318 val_318 +2008-04-09 12 318 val_318 +2008-04-09 12 409 val_409 +2008-04-09 12 470 val_470 +2008-04-09 12 137 val_137 +2008-04-09 12 369 val_369 +2008-04-09 12 316 val_316 +2008-04-09 12 169 val_169 +2008-04-09 12 413 val_413 +2008-04-09 12 85 val_85 +2008-04-09 12 77 val_77 +2008-04-09 12 0 val_0 +2008-04-09 12 490 val_490 +2008-04-09 12 87 val_87 +2008-04-09 12 364 val_364 +2008-04-09 12 179 val_179 +2008-04-09 12 118 val_118 +2008-04-09 12 134 val_134 +2008-04-09 12 395 val_395 +2008-04-09 12 282 val_282 +2008-04-09 12 138 val_138 +2008-04-09 12 238 val_238 +2008-04-09 12 419 val_419 +2008-04-09 12 15 val_15 +2008-04-09 12 118 val_118 +2008-04-09 12 72 val_72 +2008-04-09 12 90 val_90 +2008-04-09 12 307 val_307 +2008-04-09 12 19 val_19 +2008-04-09 12 435 val_435 +2008-04-09 12 10 val_10 +2008-04-09 12 277 val_277 +2008-04-09 12 273 val_273 +2008-04-09 12 306 val_306 +2008-04-09 12 224 val_224 +2008-04-09 12 309 val_309 +2008-04-09 12 389 val_389 +2008-04-09 12 327 val_327 +2008-04-09 12 242 val_242 +2008-04-09 12 369 val_369 +2008-04-09 12 392 val_392 +2008-04-09 12 272 val_272 +2008-04-09 12 331 val_331 +2008-04-09 12 401 val_401 +2008-04-09 12 242 val_242 +2008-04-09 12 452 val_452 +2008-04-09 12 177 val_177 +2008-04-09 12 226 val_226 +2008-04-09 12 5 val_5 +2008-04-09 12 497 val_497 +2008-04-09 12 402 val_402 +2008-04-09 12 396 val_396 +2008-04-09 12 317 val_317 +2008-04-09 12 395 val_395 +2008-04-09 12 58 val_58 +2008-04-09 12 35 val_35 +2008-04-09 12 336 val_336 +2008-04-09 12 95 val_95 +2008-04-09 12 11 val_11 +2008-04-09 12 168 val_168 +2008-04-09 12 34 val_34 +2008-04-09 12 229 val_229 +2008-04-09 12 233 val_233 +2008-04-09 12 143 val_143 +2008-04-09 12 472 val_472 +2008-04-09 12 322 val_322 +2008-04-09 12 498 val_498 +2008-04-09 12 160 val_160 +2008-04-09 12 195 val_195 +2008-04-09 12 42 val_42 +2008-04-09 12 321 val_321 +2008-04-09 12 430 val_430 +2008-04-09 12 119 val_119 +2008-04-09 12 489 val_489 +2008-04-09 12 458 val_458 +2008-04-09 12 78 val_78 +2008-04-09 12 76 val_76 +2008-04-09 12 41 val_41 +2008-04-09 12 223 val_223 +2008-04-09 12 492 val_492 +2008-04-09 12 149 val_149 +2008-04-09 12 449 val_449 +2008-04-09 12 218 val_218 +2008-04-09 12 228 val_228 +2008-04-09 12 138 val_138 +2008-04-09 12 453 val_453 +2008-04-09 12 30 val_30 +2008-04-09 12 209 val_209 +2008-04-09 12 64 val_64 +2008-04-09 12 468 val_468 +2008-04-09 12 76 val_76 +2008-04-09 12 74 val_74 +2008-04-09 12 342 val_342 +2008-04-09 12 69 val_69 +2008-04-09 12 230 val_230 +2008-04-09 12 33 val_33 +2008-04-09 12 368 val_368 +2008-04-09 12 103 val_103 +2008-04-09 12 296 val_296 +2008-04-09 12 113 val_113 +2008-04-09 12 216 val_216 +2008-04-09 12 367 val_367 +2008-04-09 12 344 val_344 +2008-04-09 12 167 val_167 +2008-04-09 12 274 val_274 +2008-04-09 12 219 val_219 +2008-04-09 12 239 val_239 +2008-04-09 12 485 val_485 +2008-04-09 12 116 val_116 +2008-04-09 12 223 val_223 +2008-04-09 12 256 val_256 +2008-04-09 12 263 val_263 +2008-04-09 12 70 val_70 +2008-04-09 12 487 val_487 +2008-04-09 12 480 val_480 +2008-04-09 12 401 val_401 +2008-04-09 12 288 val_288 +2008-04-09 12 191 val_191 +2008-04-09 12 5 val_5 +2008-04-09 12 244 val_244 +2008-04-09 12 438 val_438 +2008-04-09 12 128 val_128 +2008-04-09 12 467 val_467 +2008-04-09 12 432 val_432 +2008-04-09 12 202 val_202 +2008-04-09 12 316 val_316 +2008-04-09 12 229 val_229 +2008-04-09 12 469 val_469 +2008-04-09 12 463 val_463 +2008-04-09 12 280 val_280 +2008-04-09 12 2 val_2 +2008-04-09 12 35 val_35 +2008-04-09 12 283 val_283 +2008-04-09 12 331 val_331 +2008-04-09 12 235 val_235 +2008-04-09 12 80 val_80 +2008-04-09 12 44 val_44 +2008-04-09 12 193 val_193 +2008-04-09 12 321 val_321 +2008-04-09 12 335 val_335 +2008-04-09 12 104 val_104 +2008-04-09 12 466 val_466 +2008-04-09 12 366 val_366 +2008-04-09 12 175 val_175 +2008-04-09 12 403 val_403 +2008-04-09 12 483 val_483 +2008-04-09 12 53 val_53 +2008-04-09 12 105 val_105 +2008-04-09 12 257 val_257 +2008-04-09 12 406 val_406 +2008-04-09 12 409 val_409 +2008-04-09 12 190 val_190 +2008-04-09 12 406 val_406 +2008-04-09 12 401 val_401 +2008-04-09 12 114 val_114 +2008-04-09 12 258 val_258 +2008-04-09 12 90 val_90 +2008-04-09 12 203 val_203 +2008-04-09 12 262 val_262 +2008-04-09 12 348 val_348 +2008-04-09 12 424 val_424 +2008-04-09 12 12 val_12 +2008-04-09 12 396 val_396 +2008-04-09 12 201 val_201 +2008-04-09 12 217 val_217 +2008-04-09 12 164 val_164 +2008-04-09 12 431 val_431 +2008-04-09 12 454 val_454 +2008-04-09 12 478 val_478 +2008-04-09 12 298 val_298 +2008-04-09 12 125 val_125 +2008-04-09 12 431 val_431 +2008-04-09 12 164 val_164 +2008-04-09 12 424 val_424 +2008-04-09 12 187 val_187 +2008-04-09 12 382 val_382 +2008-04-09 12 5 val_5 +2008-04-09 12 70 val_70 +2008-04-09 12 397 val_397 +2008-04-09 12 480 val_480 +2008-04-09 12 291 val_291 +2008-04-09 12 24 val_24 +2008-04-09 12 351 val_351 +2008-04-09 12 255 val_255 +2008-04-09 12 104 val_104 +2008-04-09 12 70 val_70 +2008-04-09 12 163 val_163 +2008-04-09 12 438 val_438 +2008-04-09 12 119 val_119 +2008-04-09 12 414 val_414 +2008-04-09 12 200 val_200 +2008-04-09 12 491 val_491 +2008-04-09 12 237 val_237 +2008-04-09 12 439 val_439 +2008-04-09 12 360 val_360 +2008-04-09 12 248 val_248 +2008-04-09 12 479 val_479 +2008-04-09 12 305 val_305 +2008-04-09 12 417 val_417 +2008-04-09 12 199 val_199 +2008-04-09 12 444 val_444 +2008-04-09 12 120 val_120 +2008-04-09 12 429 val_429 +2008-04-09 12 169 val_169 +2008-04-09 12 443 val_443 +2008-04-09 12 323 val_323 +2008-04-09 12 325 val_325 +2008-04-09 12 277 val_277 +2008-04-09 12 230 val_230 +2008-04-09 12 478 val_478 +2008-04-09 12 178 val_178 +2008-04-09 12 468 val_468 +2008-04-09 12 310 val_310 +2008-04-09 12 317 val_317 +2008-04-09 12 333 val_333 +2008-04-09 12 493 val_493 +2008-04-09 12 460 val_460 +2008-04-09 12 207 val_207 +2008-04-09 12 249 val_249 +2008-04-09 12 265 val_265 +2008-04-09 12 480 val_480 +2008-04-09 12 83 val_83 +2008-04-09 12 136 val_136 +2008-04-09 12 353 val_353 +2008-04-09 12 172 val_172 +2008-04-09 12 214 val_214 +2008-04-09 12 462 val_462 +2008-04-09 12 233 val_233 +2008-04-09 12 406 val_406 +2008-04-09 12 133 val_133 +2008-04-09 12 175 val_175 +2008-04-09 12 189 val_189 +2008-04-09 12 454 val_454 +2008-04-09 12 375 val_375 +2008-04-09 12 401 val_401 +2008-04-09 12 421 val_421 +2008-04-09 12 407 val_407 +2008-04-09 12 384 val_384 +2008-04-09 12 256 val_256 +2008-04-09 12 26 val_26 +2008-04-09 12 134 val_134 +2008-04-09 12 67 val_67 +2008-04-09 12 384 val_384 +2008-04-09 12 379 val_379 +2008-04-09 12 18 val_18 +2008-04-09 12 462 val_462 +2008-04-09 12 492 val_492 +2008-04-09 12 100 val_100 +2008-04-09 12 298 val_298 +2008-04-09 12 9 val_9 +2008-04-09 12 341 val_341 +2008-04-09 12 498 val_498 +2008-04-09 12 146 val_146 +2008-04-09 12 458 val_458 +2008-04-09 12 362 val_362 +2008-04-09 12 186 val_186 +2008-04-09 12 285 val_285 +2008-04-09 12 348 val_348 +2008-04-09 12 167 val_167 +2008-04-09 12 18 val_18 +2008-04-09 12 273 val_273 +2008-04-09 12 183 val_183 +2008-04-09 12 281 val_281 +2008-04-09 12 344 val_344 +2008-04-09 12 97 val_97 +2008-04-09 12 469 val_469 +2008-04-09 12 315 val_315 +2008-04-09 12 84 val_84 +2008-04-09 12 28 val_28 +2008-04-09 12 37 val_37 +2008-04-09 12 448 val_448 +2008-04-09 12 152 val_152 +2008-04-09 12 348 val_348 +2008-04-09 12 307 val_307 +2008-04-09 12 194 val_194 +2008-04-09 12 414 val_414 +2008-04-09 12 477 val_477 +2008-04-09 12 222 val_222 +2008-04-09 12 126 val_126 +2008-04-09 12 90 val_90 +2008-04-09 12 169 val_169 +2008-04-09 12 403 val_403 +2008-04-09 12 400 val_400 +2008-04-09 12 200 val_200 +2008-04-09 12 97 val_97 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 4b92d167a1263..c184ebe288af4 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 @@ -53,10 +53,8 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") - ignore("partitioned table scan") { - createQueryTest("partitioned table scan", - "SELECT ds, hr, key, value FROM srcpart") - } + createQueryTest("partitioned table scan", + "SELECT ds, hr, key, value FROM srcpart") createQueryTest("hash", "SELECT hash('test') FROM src LIMIT 1") From e15e57413e07e5d4787514702f735bba0c30cae5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 26 Mar 2014 18:19:49 -0700 Subject: [PATCH 06/27] [SQL] Add a custom serializer for maps since they do not have a no-arg constructor. Author: Michael Armbrust Closes #243 from marmbrus/mapSer and squashes the following commits: 54045f7 [Michael Armbrust] Add a custom serializer for maps since they do not have a no-arg constructor. --- .../sql/execution/SparkSqlSerializer.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 1c3196ae2e7b6..915f551fb2f01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -32,6 +32,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { kryo.setRegistrationRequired(false) kryo.register(classOf[MutablePair[_, _]]) kryo.register(classOf[Array[Any]]) + kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) @@ -70,3 +71,20 @@ class BigDecimalSerializer extends Serializer[BigDecimal] { BigDecimal(input.readString()) } } + +/** + * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize + * them as `Array[(k,v)]`. + */ +class MapSerializer extends Serializer[Map[_,_]] { + def write(kryo: Kryo, output: Output, map: Map[_,_]) { + kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray) + } + + def read(kryo: Kryo, input: Input, tpe: Class[Map[_,_]]): Map[_,_] = { + kryo.readObject(input, classOf[Array[Any]]) + .sliding(2,2) + .map { case Array(k,v) => (k,v) } + .toMap + } +} From be6d96c15b3c31cd27bdd79fb259072479151ae6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Mar 2014 18:22:15 -0700 Subject: [PATCH 07/27] SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS /cc @aarondav and @andrewor14 Author: Patrick Wendell Closes #231 from pwendell/ui-binding and squashes the following commits: e8025f8 [Patrick Wendell] SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS --- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index fd638c83aac6e..ef1ad872c8ef7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -47,7 +47,8 @@ private[spark] class SparkUI( val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) + private val bindHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt private var serverInfo: Option[ServerInfo] = None @@ -79,8 +80,8 @@ private[spark] class SparkUI( /** Bind the HTTP server which backs this web interface */ def bind() { try { - serverInfo = Some(startJettyServer(host, port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) + serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) + logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) } catch { case e: Exception => logError("Failed to create Spark JettyUtils", e) @@ -111,7 +112,7 @@ private[spark] class SparkUI( logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + host + ":" + boundPort + private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort } From 3e63d98f09065386901d78c141b0da93cdce0f76 Mon Sep 17 00:00:00 2001 From: NirmalReddy Date: Wed, 26 Mar 2014 18:24:55 -0700 Subject: [PATCH 08/27] Spark 1095 : Adding explicit return types to all public methods Excluded those that are self-evident and the cases that are discussed in the mailing list. Author: NirmalReddy Author: NirmalReddy Closes #168 from NirmalReddy/Spark-1095 and squashes the following commits: ac54b29 [NirmalReddy] import misplaced 8c5ff3e [NirmalReddy] Changed syntax of unit returning methods 02d0778 [NirmalReddy] fixed explicit types in all the other packages 1c17773 [NirmalReddy] fixed explicit types in core package --- .../scala/org/apache/spark/SparkContext.scala | 31 ++++++++++++------- .../apache/spark/api/java/JavaRDDLike.scala | 15 ++++++--- .../spark/api/java/JavaSparkContext.scala | 2 +- .../apache/spark/deploy/ClientArguments.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../master/ZooKeeperPersistenceEngine.scala | 3 +- .../spark/metrics/sink/ConsoleSink.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../spark/metrics/sink/GraphiteSink.scala | 4 +-- .../org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../apache/spark/storage/StorageLevel.scala | 7 +++-- .../org/apache/spark/util/Distribution.scala | 7 +++-- .../spark/metrics/sink/GangliaSink.scala | 12 ++++--- .../scala/org/apache/spark/graphx/Graph.scala | 3 +- .../apache/spark/graphx/impl/GraphImpl.scala | 2 +- .../apache/spark/graphx/lib/Analytics.scala | 2 +- .../spark/streaming/StreamingContext.scala | 4 +-- .../streaming/api/java/JavaDStreamLike.scala | 6 ++-- .../api/java/JavaStreamingContext.scala | 22 +++++++++---- .../spark/streaming/dstream/DStream.scala | 8 +++-- .../spark/streaming/scheduler/BatchInfo.scala | 8 +++-- 25 files changed, 97 insertions(+), 57 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4dd298177f07d..b23accbbb9410 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ @@ -230,7 +231,7 @@ class SparkContext( postEnvironmentUpdate() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ - val hadoopConfiguration = { + val hadoopConfiguration: Configuration = { val env = SparkEnv.get val hadoopConf = SparkHadoopUtil.get.newConfiguration() // Explicitly check for S3 environment variables @@ -630,7 +631,7 @@ class SparkContext( * standard mutable collections. So you can use this with mutable Map, Set, etc. */ def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T] - (initialValue: R) = { + (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R,T] new Accumulable(initialValue, param) } @@ -640,7 +641,7 @@ class SparkContext( * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) + def broadcast[T](value: T): Broadcast[T] = env.broadcastManager.newBroadcast[T](value, isLocal) /** * Add a file to be downloaded with this Spark job on every node. @@ -1126,7 +1127,7 @@ object SparkContext extends Logging { implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( - rdd: RDD[(K, V)]) = + rdd: RDD[(K, V)]) = new SequenceFileRDDFunctions(rdd) implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( @@ -1163,27 +1164,33 @@ object SparkContext extends Logging { } // Helper objects for converting common types to Writable - private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = { + private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) + : WritableConverter[T] = { val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) } - implicit def intWritableConverter() = simpleWritableConverter[Int, IntWritable](_.get) + implicit def intWritableConverter(): WritableConverter[Int] = + simpleWritableConverter[Int, IntWritable](_.get) - implicit def longWritableConverter() = simpleWritableConverter[Long, LongWritable](_.get) + implicit def longWritableConverter(): WritableConverter[Long] = + simpleWritableConverter[Long, LongWritable](_.get) - implicit def doubleWritableConverter() = simpleWritableConverter[Double, DoubleWritable](_.get) + implicit def doubleWritableConverter(): WritableConverter[Double] = + simpleWritableConverter[Double, DoubleWritable](_.get) - implicit def floatWritableConverter() = simpleWritableConverter[Float, FloatWritable](_.get) + implicit def floatWritableConverter(): WritableConverter[Float] = + simpleWritableConverter[Float, FloatWritable](_.get) - implicit def booleanWritableConverter() = + implicit def booleanWritableConverter(): WritableConverter[Boolean] = simpleWritableConverter[Boolean, BooleanWritable](_.get) - implicit def bytesWritableConverter() = { + implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes) } - implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString) + implicit def stringWritableConverter(): WritableConverter[String] = + simpleWritableConverter[String, Text](_.toString) implicit def writableWritableConverter[T <: Writable]() = new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index ddac553304233..e03b8e78d5f52 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -391,19 +391,24 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Save this RDD as a text file, using string representations of elements. */ - def saveAsTextFile(path: String) = rdd.saveAsTextFile(path) + def saveAsTextFile(path: String): Unit = { + rdd.saveAsTextFile(path) + } /** * Save this RDD as a compressed text file, using string representations of elements. */ - def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) = + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit = { rdd.saveAsTextFile(path, codec) + } /** * Save this RDD as a SequenceFile of serialized objects. */ - def saveAsObjectFile(path: String) = rdd.saveAsObjectFile(path) + def saveAsObjectFile(path: String): Unit = { + rdd.saveAsObjectFile(path) + } /** * Creates tuples of the elements in this RDD by applying `f`. @@ -420,7 +425,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * executed on this RDD. It is strongly recommended that this RDD is persisted in * memory, otherwise saving it on a file will require recomputation. */ - def checkpoint() = rdd.checkpoint() + def checkpoint(): Unit = { + rdd.checkpoint() + } /** * Return whether this RDD has been checkpointed or not diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 35508b6e5acba..e531a57aced31 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -463,7 +463,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork sc.setCheckpointDir(dir) } - def getCheckpointDir = JavaUtils.optionToOptional(sc.getCheckpointDir) + def getCheckpointDir: Optional[String] = JavaUtils.optionToOptional(sc.getCheckpointDir) protected def checkpointFile[T](path: String): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 00f5cd54ad650..c07838f798799 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -112,5 +112,5 @@ private[spark] class ClientArguments(args: Array[String]) { } object ClientArguments { - def isValidJarUrl(s: String) = s.matches("(.+):(.+)jar") + def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index d2d8d6d662d55..9bdbfb33bf54f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -32,7 +32,7 @@ import scala.collection.JavaConversions._ * Contains util methods to interact with Hadoop from Spark. */ class SparkHadoopUtil { - val conf = newConfiguration() + val conf: Configuration = newConfiguration() UserGroupInformation.setConfiguration(conf) def runAsUser(user: String)(func: () => Unit) { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 5413ff671ad8d..834dfedee52ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import scala.collection.JavaConversions._ import akka.serialization.Serialization +import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} @@ -29,7 +30,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk = SparkCuratorUtil.newClient(conf) + val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) SparkCuratorUtil.mkdir(zk, WORKING_DIR) diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 4d2ffc54d8983..64eac73605388 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -38,7 +38,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry, case None => CONSOLE_DEFAULT_PERIOD } - val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { + val pollUnit: TimeUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 319f40815d65f..544848d4150b6 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -41,7 +41,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry, case None => CSV_DEFAULT_PERIOD } - val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { + val pollUnit: TimeUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 0ffdf3846dc4a..7f0a2fd16fa99 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -39,7 +39,7 @@ class GraphiteSink(val property: Properties, val registry: MetricRegistry, val GRAPHITE_KEY_UNIT = "unit" val GRAPHITE_KEY_PREFIX = "prefix" - def propertyToOption(prop: String) = Option(property.getProperty(prop)) + def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop)) if (!propertyToOption(GRAPHITE_KEY_HOST).isDefined) { throw new Exception("Graphite sink requires 'host' property.") @@ -57,7 +57,7 @@ class GraphiteSink(val property: Properties, val registry: MetricRegistry, case None => GRAPHITE_DEFAULT_PERIOD } - val pollUnit = propertyToOption(GRAPHITE_KEY_UNIT) match { + val pollUnit: TimeUnit = propertyToOption(GRAPHITE_KEY_UNIT) match { case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 8561711931047..9aa454a5c8b88 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -103,7 +103,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: array } - override val partitioner = Some(part) + override val partitioner: Some[Partitioner] = Some(part) override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { val sparkConf = SparkEnv.get.conf diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 932ff5bf369c7..3af008bd72378 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -171,7 +171,7 @@ class HadoopRDD[K, V]( array } - override def compute(theSplit: Partition, context: TaskContext) = { + override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { val iter = new NextIterator[(K, V)] { val split = theSplit.asInstanceOf[HadoopPartition] diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 8df8718f3b65b..1b503743ac117 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -116,7 +116,7 @@ class JdbcRDD[T: ClassTag]( } object JdbcRDD { - def resultSetToObjectArray(rs: ResultSet) = { + def resultSetToObjectArray(rs: ResultSet): Array[Object] = { Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index d1fff296878c3..461a749eac48b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -80,7 +80,7 @@ class NewHadoopRDD[K, V]( result } - override def compute(theSplit: Partition, context: TaskContext) = { + override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { val iter = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6af42248a5c3c..ce2b8ac27206b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -121,7 +121,7 @@ abstract class RDD[T: ClassTag]( @transient var name: String = null /** Assign a name to this RDD */ - def setName(_name: String) = { + def setName(_name: String): RDD[T] = { name = _name this } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 1b7934d59fa1d..4212a539dab4b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -126,15 +126,16 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) /** Create a new StorageLevel object */ - def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = + def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, + replication: Int = 1): StorageLevel = getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ - def apply(flags: Int, replication: Int) = + def apply(flags: Int, replication: Int): StorageLevel = getCachedStorageLevel(new StorageLevel(flags, replication)) /** Read StorageLevel object from ObjectInput stream */ - def apply(in: ObjectInput) = { + def apply(in: ObjectInput): StorageLevel = { val obj = new StorageLevel() obj.readExternal(in) getCachedStorageLevel(obj) diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index ab738c4b868fa..5b347555fe708 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -19,6 +19,8 @@ package org.apache.spark.util import java.io.PrintStream +import scala.collection.immutable.IndexedSeq + /** * Util for getting some stats from a small sample of numeric values, with some handy * summary functions. @@ -40,7 +42,8 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) * given from 0 to 1 * @param probabilities */ - def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) = { + def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) + : IndexedSeq[Double] = { probabilities.toIndexedSeq.map{p:Double => data(closestIndex(p))} } @@ -48,7 +51,7 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) math.min((p * length).toInt + startIdx, endIdx - 1) } - def showQuantiles(out: PrintStream = System.out) = { + def showQuantiles(out: PrintStream = System.out): Unit = { out.println("min\t25%\t50%\t75%\tmax") getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")} out.println diff --git a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index cd37317da77de..d03d7774e8c80 100644 --- a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.MetricRegistry import com.codahale.metrics.ganglia.GangliaReporter import info.ganglia.gmetric4j.gmetric.GMetric +import info.ganglia.gmetric4j.gmetric.GMetric.UDPAddressingMode import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem @@ -33,10 +34,10 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val GANGLIA_DEFAULT_PERIOD = 10 val GANGLIA_KEY_UNIT = "unit" - val GANGLIA_DEFAULT_UNIT = TimeUnit.SECONDS + val GANGLIA_DEFAULT_UNIT: TimeUnit = TimeUnit.SECONDS val GANGLIA_KEY_MODE = "mode" - val GANGLIA_DEFAULT_MODE = GMetric.UDPAddressingMode.MULTICAST + val GANGLIA_DEFAULT_MODE: UDPAddressingMode = GMetric.UDPAddressingMode.MULTICAST // TTL for multicast messages. If listeners are X hops away in network, must be at least X. val GANGLIA_KEY_TTL = "ttl" @@ -45,7 +46,7 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val GANGLIA_KEY_HOST = "host" val GANGLIA_KEY_PORT = "port" - def propertyToOption(prop: String) = Option(property.getProperty(prop)) + def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop)) if (!propertyToOption(GANGLIA_KEY_HOST).isDefined) { throw new Exception("Ganglia sink requires 'host' property.") @@ -58,11 +59,12 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val host = propertyToOption(GANGLIA_KEY_HOST).get val port = propertyToOption(GANGLIA_KEY_PORT).get.toInt val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL) - val mode = propertyToOption(GANGLIA_KEY_MODE) + val mode: UDPAddressingMode = propertyToOption(GANGLIA_KEY_MODE) .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE) val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt) .getOrElse(GANGLIA_DEFAULT_PERIOD) - val pollUnit = propertyToOption(GANGLIA_KEY_UNIT).map(u => TimeUnit.valueOf(u.toUpperCase)) + val pollUnit: TimeUnit = propertyToOption(GANGLIA_KEY_UNIT) + .map(u => TimeUnit.valueOf(u.toUpperCase)) .getOrElse(GANGLIA_DEFAULT_UNIT) MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 65a1a8c68f6d2..ef05623d7a0a1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -419,5 +419,6 @@ object Graph { * All the convenience operations are defined in the [[GraphOps]] class which may be * shared across multiple graph implementations. */ - implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops + implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag] + (g: Graph[VD, ED]): GraphOps[VD, ED] = g.ops } // end of Graph object diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 5e9be18990ba3..43ac11d8957f6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -197,7 +197,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], reduceFunc: (A, A) => A, - activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala index 24699dfdd38b0..fa533a512d53b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -26,7 +26,7 @@ import org.apache.spark.graphx.PartitionStrategy._ */ object Analytics extends Logging { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { val host = args(0) val taskType = args(1) val fname = args(2) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 062b888e80b48..e198c69470c1f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -431,7 +431,7 @@ class StreamingContext private[streaming] ( * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean = true) = synchronized { + def stop(stopSparkContext: Boolean = true): Unit = synchronized { scheduler.stop() logInfo("StreamingContext stopped successfully") waiter.notifyStop() @@ -489,7 +489,7 @@ object StreamingContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. */ - def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls) + def jarOfClass(cls: Class[_]): Seq[String] = SparkContext.jarOfClass(cls) private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { // Set the default cleaner delay to an hour if not already set. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index a85cd04c9319c..bb2f492d06a00 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -49,7 +49,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Print the first ten elements of each RDD generated in this DStream. This is an output * operator, so this DStream will be registered as an output stream and there materialized. */ - def print() = dstream.print() + def print(): Unit = { + dstream.print() + } /** * Return a new DStream in which each RDD has a single element generated by counting each RDD @@ -401,7 +403,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Enable periodic checkpointing of RDDs of this DStream. * @param interval Time interval after which generated RDD will be checkpointed */ - def checkpoint(interval: Duration) = { + def checkpoint(interval: Duration): DStream[T] = { dstream.checkpoint(interval) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index c48d754e439e9..b705d2ec9a58e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -477,31 +477,41 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Start the execution of the streams. */ - def start() = ssc.start() + def start(): Unit = { + ssc.start() + } /** * Wait for the execution to stop. Any exceptions that occurs during the execution * will be thrown in this thread. */ - def awaitTermination() = ssc.awaitTermination() + def awaitTermination(): Unit = { + ssc.awaitTermination() + } /** * Wait for the execution to stop. Any exceptions that occurs during the execution * will be thrown in this thread. * @param timeout time to wait in milliseconds */ - def awaitTermination(timeout: Long) = ssc.awaitTermination(timeout) + def awaitTermination(timeout: Long): Unit = { + ssc.awaitTermination(timeout) + } /** * Stop the execution of the streams. Will stop the associated JavaSparkContext as well. */ - def stop() = ssc.stop() + def stop(): Unit = { + ssc.stop() + } /** * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean) = ssc.stop(stopSparkContext) + def stop(stopSparkContext: Boolean): Unit = { + ssc.stop(stopSparkContext) + } } /** @@ -579,7 +589,7 @@ object JavaStreamingContext { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. */ - def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray + def jarOfClass(cls: Class[_]): Array[String] = SparkContext.jarOfClass(cls).toArray } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 6bff56a9d332a..d48b51aa69565 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -503,14 +503,18 @@ abstract class DStream[T: ClassTag] ( * 'this' DStream will be registered as an output stream and therefore materialized. */ @deprecated("use foreachRDD", "0.9.0") - def foreach(foreachFunc: RDD[T] => Unit) = this.foreachRDD(foreachFunc) + def foreach(foreachFunc: RDD[T] => Unit): Unit = { + this.foreachRDD(foreachFunc) + } /** * Apply a function to each RDD in this DStream. This is an output operator, so * 'this' DStream will be registered as an output stream and therefore materialized. */ @deprecated("use foreachRDD", "0.9.0") - def foreach(foreachFunc: (RDD[T], Time) => Unit) = this.foreachRDD(foreachFunc) + def foreach(foreachFunc: (RDD[T], Time) => Unit): Unit = { + this.foreachRDD(foreachFunc) + } /** * Apply a function to each RDD in this DStream. This is an output operator, so diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 4e8d07fe921fb..7f3cd2f8eb1fd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -39,17 +39,19 @@ case class BatchInfo( * was submitted to the streaming scheduler. Essentially, it is * `processingStartTime` - `submissionTime`. */ - def schedulingDelay = processingStartTime.map(_ - submissionTime) + def schedulingDelay: Option[Long] = processingStartTime.map(_ - submissionTime) /** * Time taken for the all jobs of this batch to finish processing from the time they started * processing. Essentially, it is `processingEndTime` - `processingStartTime`. */ - def processingDelay = processingEndTime.zip(processingStartTime).map(x => x._1 - x._2).headOption + def processingDelay: Option[Long] = processingEndTime.zip(processingStartTime) + .map(x => x._1 - x._2).headOption /** * Time taken for all the jobs of this batch to finish processing from the time they * were submitted. Essentially, it is `processingDelay` + `schedulingDelay`. */ - def totalDelay = schedulingDelay.zip(processingDelay).map(x => x._1 + x._2).headOption + def totalDelay: Option[Long] = schedulingDelay.zip(processingDelay) + .map(x => x._1 + x._2).headOption } From 1fa48d9422d543827011eec0cdf12d060b78a7c7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 26 Mar 2014 18:31:52 -0700 Subject: [PATCH 09/27] SPARK-1325. The maven build error for Spark Tools This is just a slight variation on https://github.com/apache/spark/pull/234 and alternative suggestion for SPARK-1325. `scala-actors` is not necessary. `SparkBuild.scala` should be updated to reflect the direct dependency on `scala-reflect` and `scala-compiler`. And the `repl` build, which has the same dependencies, should also be consistent between Maven / SBT. Author: Sean Owen Author: witgo Closes #240 from srowen/SPARK-1325 and squashes the following commits: 25bd7db [Sean Owen] Add necessary dependencies scala-reflect and scala-compiler to tools. Update repl dependencies, which are similar, to be consistent between Maven / SBT in this regard too. --- pom.xml | 5 +++++ project/SparkBuild.scala | 4 +++- repl/pom.xml | 5 +++++ tools/pom.xml | 8 ++++++++ 4 files changed, 21 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index deb89b18ada73..f0644ed284363 100644 --- a/pom.xml +++ b/pom.xml @@ -419,6 +419,11 @@ scala-compiler ${scala.version} + + org.scala-lang + scala-reflect + ${scala.version} + org.scala-lang jline diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 60f14ba37e35c..9e269e6551341 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -356,7 +356,9 @@ object SparkBuild extends Build { ) ++ assemblySettings ++ extraAssemblySettings def toolsSettings = sharedSettings ++ Seq( - name := "spark-tools" + name := "spark-tools", + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) ) ++ assemblySettings ++ extraAssemblySettings def graphxSettings = sharedSettings ++ Seq( diff --git a/repl/pom.xml b/repl/pom.xml index fc49c8b811316..78d2fe13c27eb 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -77,6 +77,11 @@ scala-compiler ${scala.version} + + org.scala-lang + scala-reflect + ${scala.version} + org.scala-lang jline diff --git a/tools/pom.xml b/tools/pom.xml index 11433e596f5b0..ae2ba64e07c21 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -55,6 +55,14 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.scala-lang + scala-reflect + + + org.scala-lang + scala-compiler + org.scalatest scalatest_${scala.binary.version} From d679843a39bb4918a08a5aebdf113ac8886a5275 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 19:30:20 -0700 Subject: [PATCH 10/27] [SPARK-1327] GLM needs to check addIntercept for intercept and weights GLM needs to check addIntercept for intercept and weights. The current implementation always uses the first weight as intercept. Added a test for training without adding intercept. JIRA: https://spark-project.atlassian.net/browse/SPARK-1327 Author: Xiangrui Meng Closes #236 from mengxr/glm and squashes the following commits: bcac1ac [Xiangrui Meng] add two tests to ensure {Lasso, Ridge}.setIntercept will throw an exceptions a104072 [Xiangrui Meng] remove protected to be compatible with 0.9 0e57aa4 [Xiangrui Meng] update Lasso and RidgeRegression to parse the weights correctly from GLM mark createModel protected mark predictPoint protected d7f629f [Xiangrui Meng] fix a bug in GLM when intercept is not used --- .../GeneralizedLinearAlgorithm.scala | 21 ++++++++------- .../apache/spark/mllib/regression/Lasso.scala | 20 +++++++++----- .../mllib/regression/LinearRegression.scala | 20 +++++++------- .../mllib/regression/RidgeRegression.scala | 18 +++++++++---- .../spark/mllib/regression/LassoSuite.scala | 9 ++++--- .../regression/LinearRegressionSuite.scala | 26 ++++++++++++++++++- .../regression/RidgeRegressionSuite.scala | 9 ++++--- 7 files changed, 86 insertions(+), 37 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index b9621530efa22..3e1ed91bf6729 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -136,25 +136,28 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features.+:(1.0))) + input.map(labeledPoint => (labeledPoint.label, 1.0 +: labeledPoint.features)) } else { input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) } val initialWeightsWithIntercept = if (addIntercept) { - initialWeights.+:(1.0) + 0.0 +: initialWeights } else { initialWeights } - val weights = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = weights(0) - val weightsScaled = weights.tail + val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val model = createModel(weightsScaled, intercept) + val (intercept, weights) = if (addIntercept) { + (weightsWithIntercept(0), weightsWithIntercept.tail) + } else { + (0.0, weightsWithIntercept) + } + + logInfo("Final weights " + weights.mkString(",")) + logInfo("Final intercept " + intercept) - logInfo("Final model weights " + model.weights.mkString(",")) - logInfo("Final model intercept " + model.intercept) - model + createModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index fb2bc9b92a51c..be63ce8538fef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -36,8 +36,10 @@ class LassoModel( extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -66,7 +68,7 @@ class LassoWithSGD private ( .setMiniBatchFraction(miniBatchFraction) // We don't want to penalize the intercept, so set this to false. - setIntercept(false) + super.setIntercept(false) var yMean = 0.0 var xColMean: DoubleMatrix = _ @@ -77,10 +79,16 @@ class LassoWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + override def setIntercept(addIntercept: Boolean): this.type = { + // TODO: Support adding intercept. + if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") + this + } + + override def createModel(weights: Array[Double], intercept: Double) = { + val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) + val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) new LassoModel(weightsScaled.data, interceptScaled) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 8ee40addb25d9..f5f15d1a33f4d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -31,13 +31,14 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LinearRegressionModel( - override val weights: Array[Double], - override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable { - - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override val weights: Array[Double], + override val intercept: Double) + extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { + + override def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -55,8 +56,7 @@ class LinearRegressionWithSGD private ( var stepSize: Double, var numIterations: Int, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LinearRegressionModel] - with Serializable { + extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { val gradient = new LeastSquaresGradient() val updater = new SimpleUpdater() @@ -69,7 +69,7 @@ class LinearRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { + override def createModel(weights: Array[Double], intercept: Double) = { new LinearRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index c504d3d40c773..feb100f21888f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -36,8 +36,10 @@ class RidgeRegressionModel( extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -67,7 +69,7 @@ class RidgeRegressionWithSGD private ( .setMiniBatchFraction(miniBatchFraction) // We don't want to penalize the intercept in RidgeRegression, so set this to false. - setIntercept(false) + super.setIntercept(false) var yMean = 0.0 var xColMean: DoubleMatrix = _ @@ -78,8 +80,14 @@ class RidgeRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + override def setIntercept(addIntercept: Boolean): this.type = { + // TODO: Support adding intercept. + if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") + this + } + + override def createModel(weights: Array[Double], intercept: Double) = { + val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) val weightsScaled = weightsMat.div(xColSd) val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 64e4cbb860f61..2cebac943e15f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -17,11 +17,8 @@ package org.apache.spark.mllib.regression - -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import org.apache.spark.SparkContext import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} class LassoSuite extends FunSuite with LocalSparkContext { @@ -104,4 +101,10 @@ class LassoSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("do not support intercept") { + intercept[UnsupportedOperationException] { + new LassoWithSGD().setIntercept(true) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 281f9df36ddb3..5d251bcbf35db 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.mllib.regression -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} @@ -57,4 +56,29 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + // Test if we can correctly learn Y = 10*X1 + 10*X2 + test("linear regression without intercept") { + val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 42), 2).cache() + val linReg = new LinearRegressionWithSGD().setIntercept(false) + linReg.optimizer.setNumIterations(1000).setStepSize(1.0) + + val model = linReg.run(testRDD) + + assert(model.intercept === 0.0) + assert(model.weights.length === 2) + assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) + assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val validationData = LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 17) + val validationRDD = sc.parallelize(validationData, 2).cache() + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 67dd06cc0f5eb..b2044ed0d8066 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,14 +17,11 @@ package org.apache.spark.mllib.regression - import org.jblas.DoubleMatrix -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} - class RidgeRegressionSuite extends FunSuite with LocalSparkContext { def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { @@ -74,4 +71,10 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { assert(ridgeErr < linearErr, "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } + + test("do not support intercept") { + intercept[UnsupportedOperationException] { + new RidgeRegressionWithSGD().setIntercept(true) + } + } } From 5b2d863e339b3955eafc0588212bd5231a9163a0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Mar 2014 08:53:42 -0700 Subject: [PATCH 11/27] Cut down the granularity of travis tests. This PR amortizes the cost of downloading all the jars and compiling core across more test cases. In one anecdotal run this change takes the cumulative time down from ~80 minutes to ~40 minutes. Author: Michael Armbrust Closes #255 from marmbrus/travis and squashes the following commits: 506b22d [Michael Armbrust] Cut down the granularity of travis tests so we can amortize the cost of compilation. --- .travis.yml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/.travis.yml b/.travis.yml index 617da2dc308cc..8ebd0d68429fc 100644 --- a/.travis.yml +++ b/.travis.yml @@ -20,18 +20,13 @@ - oraclejdk7 env: matrix: - - TEST=sql/test + - TEST="scalastyle assembly/assembly" + - TEST="catalyst/test sql/test streaming/test mllib/test graphx/test bagel/test" - TEST=hive/test - - TEST=catalyst/test - - TEST=streaming/test - - TEST=graphx/test - - TEST=mllib/test - - TEST=graphx/test - - TEST=bagel/test cache: directories: - $HOME/.m2 - $HOME/.ivy2 - $HOME/.sbt script: - - "sbt ++$TRAVIS_SCALA_VERSION scalastyle $TEST" + - "sbt ++$TRAVIS_SCALA_VERSION $TEST" From 426042ad24a54b4b776085cbf4e1896464efc613 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 27 Mar 2014 11:54:43 -0500 Subject: [PATCH 12/27] SPARK-1330 removed extra echo from comput_classpath.sh remove the extra echo which prevents spark-class from working. Note that I did not update the comment above it, which is also wrong because I'm not sure what it should do. Should hive only be included if explicitly built with sbt hive/assembly or should sbt assembly build it? Author: Thomas Graves Closes #241 from tgravescs/SPARK-1330 and squashes the following commits: b10d708 [Thomas Graves] SPARK-1330 removed extra echo from comput_classpath.sh --- bin/compute-classpath.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index d6f1ff9084c6f..bef42df71ce01 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -36,7 +36,6 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" # Hopefully we will find a way to avoid uber-jars entirely and deploy only the needed packages in # the future. if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then - echo "Hive assembly found, including hive support. If this isn't desired run sbt hive/clean." # Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost. DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}") From 53953d0933c0a7c3bd3bc1003954426363912e4b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 27 Mar 2014 11:49:11 -0700 Subject: [PATCH 13/27] SPARK-1335. Also increase perm gen / code cache for scalatest when invoked via Maven build I am observing build failures when the Maven build reaches tests in the new SQL components. (I'm on Java 7 / OSX 10.9). The failure is the usual complaint from scala, that it's out of permgen space, or that JIT out of code cache space. I see that various build scripts increase these both for SBT. This change simply adds these settings to scalatest's arguments. Works for me and seems a bit more consistent. (I also snuck in cures for new build warnings from new scaladoc. Felt too trivial for a new PR, although it's separate. Just something I also saw while examining the build output.) Author: Sean Owen Closes #253 from srowen/SPARK-1335 and squashes the following commits: c0f2d31 [Sean Owen] Appease scalastyle with a newline at the end of the file a02679c [Sean Owen] Fix scaladoc errors due to missing links, which are generating build warnings, from some recent doc changes. We apparently can't generate links outside the module. b2c6a09 [Sean Owen] Add perm gen, code cache settings to scalatest, mirroring SBT settings elsewhere, which allows tests to complete in at least one environment where they are failing. (Also removed a duplicate -Xms setting elsewhere.) --- pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index f0644ed284363..72acf2b402703 100644 --- a/pom.xml +++ b/pom.xml @@ -646,7 +646,6 @@ -deprecation - -Xms64m -Xms1024m -Xmx1024m -XX:PermSize=${PermGen} @@ -689,7 +688,7 @@ ${project.build.directory}/surefire-reports . ${project.build.directory}/SparkTestSuite.txt - -Xms64m -Xmx3g + -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m From 6f986f0b87bd03f4df2bf6c917e61241e9b14ac2 Mon Sep 17 00:00:00 2001 From: Petko Nikolov Date: Thu, 27 Mar 2014 15:49:07 -0700 Subject: [PATCH 14/27] [SPARK-1268] Adding XOR and AND-NOT operations to spark.util.collection.BitSet Symmetric difference (xor) in particular is useful for computing some distance metrics (e.g. Hamming). Unit tests added. Author: Petko Nikolov Closes #172 from petko-nikolov/bitset-imprv and squashes the following commits: 451f28b [Petko Nikolov] fixed style mistakes 5beba18 [Petko Nikolov] rm outer loop in andNot test 0e61035 [Petko Nikolov] conform to spark style; rm redundant asserts; more unit tests added; use arraycopy instead of loop d53cdb9 [Petko Nikolov] rm incidentally added space 4e1df43 [Petko Nikolov] adding xor and and-not to BitSet; unit tests added --- .../apache/spark/util/collection/BitSet.scala | 39 +++++++++ .../spark/util/collection/BitSetSuite.scala | 83 +++++++++++++++++++ 2 files changed, 122 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index d3153d2cac4a5..af1f64649f354 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -88,6 +88,45 @@ class BitSet(numBits: Int) extends Serializable { newBS } + /** + * Compute the symmetric difference by performing bit-wise XOR of the two sets returning the + * result. + */ + def ^(other: BitSet): BitSet = { + val newBS = new BitSet(math.max(capacity, other.capacity)) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while (ind < smaller) { + newBS.words(ind) = words(ind) ^ other.words(ind) + ind += 1 + } + if (ind < numWords) { + Array.copy( words, ind, newBS.words, ind, numWords - ind ) + } + if (ind < other.numWords) { + Array.copy( other.words, ind, newBS.words, ind, other.numWords - ind ) + } + newBS + } + + /** + * Compute the difference of the two sets by performing bit-wise AND-NOT returning the + * result. + */ + def andNot(other: BitSet): BitSet = { + val newBS = new BitSet(capacity) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while (ind < smaller) { + newBS.words(ind) = words(ind) & ~other.words(ind) + ind += 1 + } + if (ind < numWords) { + Array.copy( words, ind, newBS.words, ind, numWords - ind ) + } + newBS + } + /** * Sets the bit at the specified index to true. * @param index the bit index diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index c32183c134f9c..b85a409a4b2e9 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -69,4 +69,87 @@ class BitSetSuite extends FunSuite { assert(bitset.nextSetBit(96) === 96) assert(bitset.nextSetBit(97) === -1) } + + test( "xor len(bitsetX) < len(bitsetY)" ) { + val setBitsX = Seq( 0, 2, 3, 37, 41 ) + val setBitsY = Seq( 0, 1, 3, 37, 38, 41, 85) + val bitsetX = new BitSet(60) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(100) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetXor = bitsetX ^ bitsetY + + assert(bitsetXor.nextSetBit(0) === 1) + assert(bitsetXor.nextSetBit(1) === 1) + assert(bitsetXor.nextSetBit(2) === 2) + assert(bitsetXor.nextSetBit(3) === 38) + assert(bitsetXor.nextSetBit(38) === 38) + assert(bitsetXor.nextSetBit(39) === 85) + assert(bitsetXor.nextSetBit(42) === 85) + assert(bitsetXor.nextSetBit(85) === 85) + assert(bitsetXor.nextSetBit(86) === -1) + + } + + test( "xor len(bitsetX) > len(bitsetY)" ) { + val setBitsX = Seq( 0, 1, 3, 37, 38, 41, 85) + val setBitsY = Seq( 0, 2, 3, 37, 41 ) + val bitsetX = new BitSet(100) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(60) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetXor = bitsetX ^ bitsetY + + assert(bitsetXor.nextSetBit(0) === 1) + assert(bitsetXor.nextSetBit(1) === 1) + assert(bitsetXor.nextSetBit(2) === 2) + assert(bitsetXor.nextSetBit(3) === 38) + assert(bitsetXor.nextSetBit(38) === 38) + assert(bitsetXor.nextSetBit(39) === 85) + assert(bitsetXor.nextSetBit(42) === 85) + assert(bitsetXor.nextSetBit(85) === 85) + assert(bitsetXor.nextSetBit(86) === -1) + + } + + test( "andNot len(bitsetX) < len(bitsetY)" ) { + val setBitsX = Seq( 0, 2, 3, 37, 41, 48 ) + val setBitsY = Seq( 0, 1, 3, 37, 38, 41, 85) + val bitsetX = new BitSet(60) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(100) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetDiff = bitsetX.andNot( bitsetY ) + + assert(bitsetDiff.nextSetBit(0) === 2) + assert(bitsetDiff.nextSetBit(1) === 2) + assert(bitsetDiff.nextSetBit(2) === 2) + assert(bitsetDiff.nextSetBit(3) === 48) + assert(bitsetDiff.nextSetBit(48) === 48) + assert(bitsetDiff.nextSetBit(49) === -1) + assert(bitsetDiff.nextSetBit(65) === -1) + } + + test( "andNot len(bitsetX) > len(bitsetY)" ) { + val setBitsX = Seq( 0, 1, 3, 37, 38, 41, 85) + val setBitsY = Seq( 0, 2, 3, 37, 41, 48 ) + val bitsetX = new BitSet(100) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(60) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetDiff = bitsetX.andNot( bitsetY ) + + assert(bitsetDiff.nextSetBit(0) === 1) + assert(bitsetDiff.nextSetBit(1) === 1) + assert(bitsetDiff.nextSetBit(2) === 38) + assert(bitsetDiff.nextSetBit(3) === 38) + assert(bitsetDiff.nextSetBit(38) === 38) + assert(bitsetDiff.nextSetBit(39) === 85) + assert(bitsetDiff.nextSetBit(85) === 85) + assert(bitsetDiff.nextSetBit(86) === -1) + } } From 3d89043b7ed13bc1bb703f6eb7c00e46b936de1e Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 27 Mar 2014 22:17:15 -0700 Subject: [PATCH 15/27] [SPARK-1210] Prevent ContextClassLoader of Actor from becoming ClassLoader of Executo... ...r. Constructor of `org.apache.spark.executor.Executor` should not set context class loader of current thread, which is backend Actor's thread. Run the following code in local-mode REPL. ``` scala> case class Foo(i: Int) scala> val ret = sc.parallelize((1 to 100).map(Foo), 10).collect ``` This causes errors as follows: ``` ERROR actor.OneForOneStrategy: [L$line5.$read$$iwC$$iwC$$iwC$$iwC$Foo; java.lang.ArrayStoreException: [L$line5.$read$$iwC$$iwC$$iwC$$iwC$Foo; at scala.runtime.ScalaRunTime$.array_update(ScalaRunTime.scala:88) at org.apache.spark.SparkContext$$anonfun$runJob$3.apply(SparkContext.scala:870) at org.apache.spark.SparkContext$$anonfun$runJob$3.apply(SparkContext.scala:870) at org.apache.spark.scheduler.JobWaiter.taskSucceeded(JobWaiter.scala:56) at org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:859) at org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:616) at org.apache.spark.scheduler.DAGScheduler$$anonfun$start$1$$anon$2$$anonfun$receive$1.applyOrElse(DAGScheduler.scala:207) at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498) at akka.actor.ActorCell.invoke(ActorCell.scala:456) at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237) at akka.dispatch.Mailbox.run(Mailbox.scala:219) at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386) at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) ``` This is because the class loaders to deserialize result `Foo` instances might be different from backend Actor's, and the Actor's class loader should be the same as Driver's. Author: Takuya UESHIN Closes #15 from ueshin/wip/wrongcontextclassloader and squashes the following commits: d79e8c0 [Takuya UESHIN] Change a parent class loader of ExecutorURLClassLoader. c6c09b6 [Takuya UESHIN] Add a test to collect objects of class defined in repl. 43e0feb [Takuya UESHIN] Prevent ContextClassLoader of Actor from becoming ClassLoader of Executor. --- .../scala/org/apache/spark/executor/Executor.scala | 5 ++--- .../test/scala/org/apache/spark/repl/ReplSuite.scala | 11 +++++++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 8fe9b848ba145..13e2e292428b4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -112,11 +112,10 @@ private[spark] class Executor( } } - // Create our ClassLoader and set it on this thread + // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager private val urlClassLoader = createClassLoader() private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) - Thread.currentThread.setContextClassLoader(replClassLoader) // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. @@ -294,7 +293,7 @@ private[spark] class Executor( * created by the interpreter to the search path */ private def createClassLoader(): ExecutorURLClassLoader = { - val loader = this.getClass.getClassLoader + val loader = Thread.currentThread().getContextClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 8203b8f6122e1..4155007c6d337 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -242,4 +242,15 @@ class ReplSuite extends FunSuite { assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) } } + + test("collecting objects of class defined in repl") { + val output = runInterpreter("local[2]", + """ + |case class Foo(i: Int) + |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("ret: Array[Foo] = Array(Foo(1),", output) + } } From 632c322036b123c6f72e0c8b87d50e08bec3a1ab Mon Sep 17 00:00:00 2001 From: Nick Lanham Date: Thu, 27 Mar 2014 22:45:00 -0700 Subject: [PATCH 16/27] Make sed do -i '' on OSX I don't have access to an OSX machine, so if someone could test this that would be great. Author: Nick Lanham Closes #258 from nicklan/osx-sed-fix and squashes the following commits: a6f158f [Nick Lanham] Also make mktemp work on OSX 558fd6e [Nick Lanham] Make sed do -i '' on OSX --- make-distribution.sh | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index 6bc6819d8da92..b07aef2ef49c2 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -128,7 +128,7 @@ if [ "$SPARK_TACHYON" == "true" ]; then TACHYON_VERSION="0.4.1" TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/tachyon-${TACHYON_VERSION}-bin.tar.gz" - TMPD=`mktemp -d` + TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` pushd $TMPD > /dev/null echo "Fetchting tachyon tgz" @@ -139,7 +139,14 @@ if [ "$SPARK_TACHYON" == "true" ]; then mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" - sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + + if [[ `uname -a` == Darwin* ]]; then + # osx sed wants an empty argument to -i option of sed + TACHYON_SED="sed -i ''" + else + TACHYON_SED="sed -i" + fi + $TACHYON_SED -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" popd > /dev/null rm -rf $TMPD From 60abc252545ec7a5d59957a32e764cd18f6c16b4 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 28 Mar 2014 00:21:49 -0700 Subject: [PATCH 17/27] SPARK-1096, a space after comment start style checker. Author: Prashant Sharma Closes #124 from ScrapCodes/SPARK-1096/scalastyle-comment-check and squashes the following commits: 214135a [Prashant Sharma] Review feedback. 5eba88c [Prashant Sharma] Fixed style checks for ///+ comments. e54b2f8 [Prashant Sharma] improved message, work around. 83e7144 [Prashant Sharma] removed dependency on scalastyle in plugin, since scalastyle sbt plugin already depends on the right version. Incase we update the plugin we will have to adjust our spark-style project to depend on right scalastyle version. 810a1d6 [Prashant Sharma] SPARK-1096, a space after comment style checker. ba33193 [Prashant Sharma] scala style as a project --- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../spark/deploy/LocalSparkCluster.scala | 4 +- .../deploy/master/LeaderElectionAgent.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 1 - .../apache/spark/metrics/MetricsConfig.scala | 2 +- .../org/apache/spark/network/Connection.scala | 18 +++--- .../spark/network/ConnectionManager.scala | 18 +++--- .../spark/network/ConnectionManagerTest.scala | 4 +- .../apache/spark/network/ReceiverTest.scala | 2 +- .../org/apache/spark/network/SenderTest.scala | 2 +- .../spark/network/netty/FileHeader.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/util/ClosureCleaner.scala | 10 ++-- .../util/IndestructibleActorSystem.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 4 +- .../org/apache/spark/AccumulatorSuite.scala | 6 +- .../org/apache/spark/CheckpointSuite.scala | 1 - .../org/apache/spark/PartitioningSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 2 +- .../org/apache/spark/examples/LocalALS.scala | 1 - .../examples/SimpleSkewedGroupByTest.scala | 2 +- .../org/apache/spark/examples/SparkALS.scala | 1 - .../apache/spark/examples/SparkHdfsLR.scala | 2 - .../streaming/examples/ActorWordCount.scala | 2 +- .../streaming/examples/ZeroMQWordCount.scala | 2 +- .../streaming/zeromq/ZeroMQReceiver.scala | 2 +- .../org/apache/spark/graphx/EdgeTriplet.scala | 4 +- .../apache/spark/graphx/impl/GraphImpl.scala | 4 +- .../graphx/impl/MessageToPartition.scala | 2 +- .../spark/graphx/impl/Serializers.scala | 2 - .../spark/graphx/util/BytecodeUtils.scala | 2 +- .../spark/graphx/util/GraphGenerators.scala | 2 +- project/SparkBuild.scala | 6 +- project/plugins.sbt | 1 + project/project/SparkPluginBuild.scala | 44 +++++++++++++++ .../SparkSpaceAfterCommentStyleCheck.scala | 56 +++++++++++++++++++ .../spark/repl/ExecutorClassLoader.scala | 4 +- .../org/apache/spark/repl/SparkIMain.scala | 2 +- scalastyle-config.xml | 1 + .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../apache/spark/sql/hive/TableReader.scala | 2 +- .../apache/spark/streaming/DStreamGraph.scala | 2 +- .../dstream/NetworkInputDStream.scala | 3 +- .../dstream/ReducedWindowedDStream.scala | 4 +- .../streaming/dstream/StateDStream.scala | 5 +- .../scheduler/NetworkInputTracker.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 +- .../spark/streaming/InputStreamsSuite.scala | 3 +- .../apache/spark/deploy/yarn/ClientBase.scala | 1 - .../yarn/ClientDistributedCacheManager.scala | 4 +- .../ClientDistributedCacheManagerSuite.scala | 2 +- 55 files changed, 180 insertions(+), 88 deletions(-) create mode 100644 project/project/SparkPluginBuild.scala create mode 100644 project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index a1af63fa4a391..5ceac28fe7afb 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -81,7 +81,7 @@ class SparkEnv private[spark] ( // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. - //actorSystem.awaitTermination() + // actorSystem.awaitTermination() } private[spark] diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 3cd71213769b7..2595c15104e87 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -167,7 +167,7 @@ extends Logging { private var initialized = false private var conf: SparkConf = null def initialize(_isDriver: Boolean, conf: SparkConf) { - TorrentBroadcast.conf = conf //TODO: we might have to fix it in tests + TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests synchronized { if (!initialized) { initialized = true diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index a73b459c3cea1..9a7a113c95715 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -66,9 +66,9 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I // TODO: In Akka 2.1.x, ActorSystem.awaitTermination hangs when you have remote actors! // This is unfortunate, but for now we just comment it out. workerActorSystems.foreach(_.shutdown()) - //workerActorSystems.foreach(_.awaitTermination()) + // workerActorSystems.foreach(_.awaitTermination()) masterActorSystems.foreach(_.shutdown()) - //masterActorSystems.foreach(_.awaitTermination()) + // masterActorSystems.foreach(_.awaitTermination()) masterActorSystems.clear() workerActorSystems.clear() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala index a730fe1f599af..4433a2ec29be6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -30,7 +30,7 @@ import org.apache.spark.deploy.master.MasterMessages.ElectedLeader * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]] */ private[spark] trait LeaderElectionAgent extends Actor { - //TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring. + // TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring. val masterActor: ActorRef } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 13e2e292428b4..aecb069e4202b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -275,7 +275,6 @@ private[spark] class Executor( // have left some weird state around depending on when the exception was thrown, but on // the other hand, maybe we could detect that when future tasks fail and exit then. logError("Exception in task ID " + taskId, t) - //System.exit(1) } } finally { // TODO: Unregister shuffle memory only for ResultTask diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 6883a54494598..3e3e18c3537d0 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -42,7 +42,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi } def initialize() { - //Add default properties in case there's no properties file + // Add default properties in case there's no properties file setDefaultProperties(properties) // If spark.metrics.conf is not set, try to get file in class path diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 8fd9c2b87d256..16bd00fd189ff 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -48,7 +48,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, channel.socket.setTcpNoDelay(true) channel.socket.setReuseAddress(true) channel.socket.setKeepAlive(true) - /*channel.socket.setReceiveBufferSize(32768) */ + /* channel.socket.setReceiveBufferSize(32768) */ @volatile private var closed = false var onCloseCallback: Connection => Unit = null @@ -206,12 +206,12 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, private class Outbox { val messages = new Queue[Message]() - val defaultChunkSize = 65536 //32768 //16384 + val defaultChunkSize = 65536 var nextMessageToBeUsed = 0 def addMessage(message: Message) { messages.synchronized{ - /*messages += message*/ + /* messages += message*/ messages.enqueue(message) logDebug("Added [" + message + "] to outbox for sending to " + "[" + getRemoteConnectionManagerId() + "]") @@ -221,8 +221,8 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, def getChunk(): Option[MessageChunk] = { messages.synchronized { while (!messages.isEmpty) { - /*nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ - /*val message = messages(nextMessageToBeUsed)*/ + /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ + /* val message = messages(nextMessageToBeUsed)*/ val message = messages.dequeue val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { @@ -262,7 +262,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, val currentBuffers = new ArrayBuffer[ByteBuffer]() - /*channel.socket.setSendBufferSize(256 * 1024)*/ + /* channel.socket.setSendBufferSize(256 * 1024)*/ override def getRemoteAddress() = address @@ -355,7 +355,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } case None => { // changeConnectionKeyInterest(0) - /*key.interestOps(0)*/ + /* key.interestOps(0)*/ return false } } @@ -540,10 +540,10 @@ private[spark] class ReceivingConnection( return false } - /*logDebug("Read " + bytesRead + " bytes for the buffer")*/ + /* logDebug("Read " + bytesRead + " bytes for the buffer")*/ if (currentChunk.buffer.remaining == 0) { - /*println("Filled buffer at " + System.currentTimeMillis)*/ + /* println("Filled buffer at " + System.currentTimeMillis)*/ val bufferMessage = inbox.getMessageForChunk(currentChunk).get if (bufferMessage.isCompletelyReceived) { bufferMessage.flip diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index a75130cba2a2e..2682f9d0ed7f0 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -505,7 +505,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } handleMessageExecutor.execute(runnable) - /*handleMessage(connection, message)*/ + /* handleMessage(connection, message)*/ } private def handleClientAuthentication( @@ -733,7 +733,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, logTrace("Sending Security [" + message + "] to [" + connManagerId + "]") val connection = connectionsById.getOrElseUpdate(connManagerId, startNewConnection()) - //send security message until going connection has been authenticated + // send security message until going connection has been authenticated connection.send(message) wakeupSelector() @@ -859,14 +859,14 @@ private[spark] object ConnectionManager { None }) - /*testSequentialSending(manager)*/ - /*System.gc()*/ + /* testSequentialSending(manager)*/ + /* System.gc()*/ - /*testParallelSending(manager)*/ - /*System.gc()*/ + /* testParallelSending(manager)*/ + /* System.gc()*/ - /*testParallelDecreasingSending(manager)*/ - /*System.gc()*/ + /* testParallelDecreasingSending(manager)*/ + /* System.gc()*/ testContinuousSending(manager) System.gc() @@ -948,7 +948,7 @@ private[spark] object ConnectionManager { val ms = finishTime - startTime val tput = mb * 1000.0 / ms println("--------------------------") - /*println("Started at " + startTime + ", finished at " + finishTime) */ + /* println("Started at " + startTime + ", finished at " + finishTime) */ println("Sent " + mb + " MB in " + ms + " ms (" + tput + " MB/s)") println("--------------------------") println() diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 35f64134b073a..e5745d7daa153 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -47,8 +47,8 @@ private[spark] object ConnectionManagerTest extends Logging{ val slaves = slavesFile.mkString.split("\n") slavesFile.close() - /*println("Slaves")*/ - /*slaves.foreach(println)*/ + /* println("Slaves")*/ + /* slaves.foreach(println)*/ val tasknum = if (args.length > 2) args(2).toInt else slaves.length val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 3c09a713c6fe0..17fd931c9f075 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -27,7 +27,7 @@ private[spark] object ReceiverTest { println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - /*println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ + /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ val buffer = ByteBuffer.wrap("response".getBytes) Some(Message.createBufferMessage(buffer, msg.id)) }) diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index aac2c24a46faa..905eddfbb9450 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -50,7 +50,7 @@ private[spark] object SenderTest { (0 until count).foreach(i => { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis - /*println("Started timer at " + startTime)*/ + /* println("Started timer at " + startTime)*/ val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index f9082ffb9141a..4164e81d3a8ae 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -32,7 +32,7 @@ private[spark] class FileHeader ( buf.writeInt(fileLen) buf.writeInt(blockId.name.length) blockId.name.foreach((x: Char) => buf.writeByte(x)) - //padding the rest of header + // padding the rest of header if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) } else { 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 77c558ac46f6f..4fce47e1ee8de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -753,7 +753,7 @@ class DAGScheduler( val properties = if (stageIdToActiveJob.contains(jobId)) { stageIdToActiveJob(stage.jobId).properties } else { - //this stage will be assigned to "default" pool + // this stage will be assigned to "default" pool null } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 990e01a3e7959..7bfc30b4208a3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -172,7 +172,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A properties += ((key, value)) } } - //TODO (prashant) send conf instead of properties + // TODO (prashant) send conf instead of properties driverActor = actorSystem.actorOf( Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index bcfc39146a61e..2fbbda5b76c74 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -284,7 +284,7 @@ object BlockFetcherIterator { } } catch { case x: InterruptedException => logInfo("Copier Interrupted") - //case _ => throw new SparkException("Exception Throw in Shuffle Copier") + // case _ => throw new SparkException("Exception Throw in Shuffle Copier") } } } diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index a8d20ee332355..cdbbc65292188 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -112,7 +112,7 @@ private[spark] object ClosureCleaner extends Logging { accessedFields(cls) = Set[String]() for (cls <- func.getClass :: innerClasses) getClassReader(cls).accept(new FieldAccessFinder(accessedFields), 0) - //logInfo("accessedFields: " + accessedFields) + // logInfo("accessedFields: " + accessedFields) val inInterpreter = { try { @@ -139,13 +139,13 @@ private[spark] object ClosureCleaner extends Logging { val field = cls.getDeclaredField(fieldName) field.setAccessible(true) val value = field.get(obj) - //logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); + // logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); field.set(outer, value) } } if (outer != null) { - //logInfo("2: Setting $outer on " + func.getClass + " to " + outer); + // logInfo("2: Setting $outer on " + func.getClass + " to " + outer); val field = func.getClass.getDeclaredField("$outer") field.setAccessible(true) field.set(func, outer) @@ -153,7 +153,7 @@ private[spark] object ClosureCleaner extends Logging { } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { - //logInfo("Creating a " + cls + " with outer = " + outer) + // logInfo("Creating a " + cls + " with outer = " + outer) if (!inInterpreter) { // This is a bona fide closure class, whose constructor has no effects // other than to set its fields, so use its constructor @@ -170,7 +170,7 @@ private[spark] object ClosureCleaner extends Logging { val newCtor = rf.newConstructorForSerialization(cls, parentCtor) val obj = newCtor.newInstance().asInstanceOf[AnyRef] if (outer != null) { - //logInfo("3: Setting $outer on " + cls + " to " + outer); + // logInfo("3: Setting $outer on " + cls + " to " + outer); val field = cls.getDeclaredField("$outer") field.setAccessible(true) field.set(obj, outer) diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala index c539d2f708f95..4188a869c13da 100644 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala @@ -49,7 +49,7 @@ private[akka] class IndestructibleActorSystemImpl( if (isFatalError(cause) && !settings.JvmExitOnFatalError) { log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) - //shutdown() //TODO make it configurable + // shutdown() //TODO make it configurable } else { fallbackHandler.uncaughtException(thread, cause) } diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index 2c1a6f8fd0a44..a898824cff0ca 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -24,8 +24,8 @@ package org.apache.spark.util * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ -case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T1, - @specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2] +case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T1, + @specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T2] (var _1: T1, var _2: T2) extends Product2[T1, T2] { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 6c73ea6949dd2..4e7c34e6d1ada 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -66,7 +66,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte test ("add value to collection accumulators") { val maxI = 1000 - for (nThreads <- List(1, 10)) { //test single & multi-threaded + for (nThreads <- List(1, 10)) { // test single & multi-threaded 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) @@ -83,7 +83,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte test ("value not readable in tasks") { val maxI = 1000 - for (nThreads <- List(1, 10)) { //test single & multi-threaded + for (nThreads <- List(1, 10)) { // test single & multi-threaded 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) @@ -124,7 +124,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte test ("localValue readable in tasks") { val maxI = 1000 - for (nThreads <- List(1, 10)) { //test single & multi-threaded + for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) val groupedInts = (1 to (maxI/20)).map {x => (20 * (x - 1) to 20 * x).toSet} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d2e29f20f0b08..d2555b7c052c1 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -432,7 +432,6 @@ object CheckpointSuite { // This is a custom cogroup function that does not use mapValues like // the PairRDDFunctions.cogroup() def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = { - //println("First = " + first + ", second = " + second) new CoGroupedRDD[K]( Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 996db70809320..7c30626a0c421 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -146,7 +146,7 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(intercept[SparkException]{ arrs.distinct() }.getMessage.contains("array")) // We can't catch all usages of arrays, since they might occur inside other collections: - //assert(fails { arrPairs.distinct() }) + // assert(fails { arrPairs.distinct() }) assert(intercept[SparkException]{ arrPairs.partitionBy(new HashPartitioner(2)) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.join(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.leftOuterJoin(arrPairs) }.getMessage.contains("array")) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index a25ce35736146..7c843772bc2e0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -111,7 +111,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) - //just to make sure some of the tasks take a noticeable amount of time + // just to make sure some of the tasks take a noticeable amount of time val w = {i:Int => if (i == 0) Thread.sleep(100) diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index eb8f5915605de..616214fb5e3a6 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -39,7 +39,7 @@ class UtilsSuite extends FunSuite { } test("copyStream") { - //input array initialization + // input array initialization val bytes = Array.ofDim[Byte](9000) Random.nextBytes(bytes) diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index c8ecbb8e41a86..0095cb8425456 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -53,7 +53,6 @@ object LocalALS { for (i <- 0 until M; j <- 0 until U) { r.set(i, j, blas.ddot(ms(i), us(j))) } - //println("R: " + r) blas.daxpy(-1, targetR, r) val sumSqs = r.aggregate(Functions.plus, Functions.square) sqrt(sumSqs / (M * U)) diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 73b0e216cac98..1fdb324b89f3a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -61,7 +61,7 @@ object SimpleSkewedGroupByTest { println("RESULT: " + pairs1.groupByKey(numReducers).count) // Print how many keys each reducer got (for debugging) - //println("RESULT: " + pairs1.groupByKey(numReducers) + // println("RESULT: " + pairs1.groupByKey(numReducers) // .map{case (k,v) => (k, v.size)} // .collectAsMap) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index ce4b3c8451e00..f59ab7e7cc24a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -54,7 +54,6 @@ object SparkALS { for (i <- 0 until M; j <- 0 until U) { r.set(i, j, blas.ddot(ms(i), us(j))) } - //println("R: " + r) blas.daxpy(-1, targetR, r) val sumSqs = r.aggregate(Functions.plus, Functions.square) sqrt(sumSqs / (M * U)) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index cf1fc3e808c76..e698b9bf376e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -34,8 +34,6 @@ object SparkHdfsLR { case class DataPoint(x: Vector, y: Double) def parsePoint(line: String): DataPoint = { - //val nums = line.split(' ').map(_.toDouble) - //return DataPoint(new Vector(nums.slice(1, D+1)), nums(0)) val tok = new java.util.StringTokenizer(line, " ") var y = tok.nextToken.toDouble var x = new Array[Double](D) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 62d3a52615584..a22e64ca3ce45 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -168,7 +168,7 @@ object ActorWordCount { Props(new SampleActorReceiver[String]("akka.tcp://test@%s:%s/user/FeederActor".format( host, port.toInt))), "SampleReceiver") - //compute wordcount + // compute wordcount lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print() ssc.start() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 35be7ffa1e872..35f8f885f8f0e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -88,7 +88,7 @@ object ZeroMQWordCount { def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator - //For this stream, a zeroMQ publisher should be running. + // For this stream, a zeroMQ publisher should be running. val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index 6acba25f44c0a..a538c38dc4d6f 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -44,7 +44,7 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, case m: ZMQMessage => logDebug("Received message for:" + m.frame(0)) - //We ignore first frame for processing as it is the topic + // We ignore first frame for processing as it is the topic val bytes = m.frames.tail pushBlock(bytesToObjects(bytes)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index fea43c3b2bbf1..dfc6a801587d2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -27,12 +27,12 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { /** * The source vertex attribute */ - var srcAttr: VD = _ //nullValue[VD] + var srcAttr: VD = _ // nullValue[VD] /** * The destination vertex attribute */ - var dstAttr: VD = _ //nullValue[VD] + var dstAttr: VD = _ // nullValue[VD] /** * Set the edge properties of this triplet. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 43ac11d8957f6..c2b510a31ee3f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -190,9 +190,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } - ////////////////////////////////////////////////////////////////////////////////////////////////// + // /////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods - ////////////////////////////////////////////////////////////////////////////////////////////////// + // /////////////////////////////////////////////////////////////////////////////////////////////// override def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index fe6fe76defdc5..bebe3740bc6c0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -45,7 +45,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( * @param data value to send */ private[graphx] -class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( +class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T]( @transient var partition: PartitionID, var data: T) extends Product2[PartitionID, T] with Serializable { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 34a145e01818f..2f2c524df6394 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -298,7 +298,6 @@ abstract class ShuffleSerializationStream(s: OutputStream) extends Serialization s.write(v.toInt) } - //def writeDouble(v: Double): Unit = writeUnsignedVarLong(java.lang.Double.doubleToLongBits(v)) def writeDouble(v: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(v)) override def flush(): Unit = s.flush() @@ -391,7 +390,6 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat (s.read() & 0xFF) } - //def readDouble(): Double = java.lang.Double.longBitsToDouble(readUnsignedVarLong()) def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) override def close(): Unit = s.close() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index 014a7335f85cc..087b1156f690b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -65,7 +65,7 @@ private[graphx] object BytecodeUtils { val finder = new MethodInvocationFinder(c.getName, m) getClassReader(c).accept(finder, 0) for (classMethod <- finder.methodsInvoked) { - //println(classMethod) + // println(classMethod) if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { return true } else if (!seen.contains(classMethod)) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index f841846c0e510..a3c8de3f9068f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -123,7 +123,7 @@ object GraphGenerators { * the dimensions of the adjacency matrix */ private def addEdge(numVertices: Int): Edge[Int] = { - //val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0) + // val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0) val v = math.round(numVertices.toFloat/2.0).toInt val (src, dst) = chooseCell(v, v, v) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9e269e6551341..2549bc9710f1f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,7 +17,7 @@ import sbt._ import sbt.Classpaths.publishTask -import Keys._ +import sbt.Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ import scala.util.Properties @@ -27,7 +27,7 @@ import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact import scala.collection.JavaConversions._ // For Sonatype publishing -//import com.jsuereth.pgp.sbtplugin.PgpKeys._ +// import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { val SPARK_VERSION = "1.0.0-SNAPSHOT" @@ -200,7 +200,7 @@ object SparkBuild extends Build { publishMavenStyle := true, - //useGpg in Global := true, + // useGpg in Global := true, pomExtra := ( diff --git a/project/plugins.sbt b/project/plugins.sbt index 4ff6f67af45c0..5aa8a1ec2409b 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -22,3 +22,4 @@ 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") + diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala new file mode 100644 index 0000000000000..43361aa2b4c41 --- /dev/null +++ b/project/project/SparkPluginBuild.scala @@ -0,0 +1,44 @@ +/* + * 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. + */ + +import sbt._ +import sbt.Keys._ + +/** + * This plugin project is there to define new scala style rules for spark. This is + * a plugin project so that this gets compiled first and is put on the classpath and + * becomes available for scalastyle sbt plugin. + */ +object SparkPluginDef extends Build { + lazy val root = Project("plugins", file(".")) dependsOn(sparkStyle) + lazy val sparkStyle = Project("spark-style", file("spark-style"), settings = styleSettings) + val sparkVersion = "1.0.0-SNAPSHOT" + // There is actually no need to publish this artifact. + def styleSettings = Defaults.defaultSettings ++ Seq ( + name := "spark-style", + organization := "org.apache.spark", + version := sparkVersion, + scalaVersion := "2.10.3", + scalacOptions := Seq("-unchecked", "-deprecation"), + libraryDependencies ++= Dependencies.scalaStyle, + sbtPlugin := true + ) + + object Dependencies { + val scalaStyle = Seq("org.scalastyle" %% "scalastyle" % "0.4.0") + } +} diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala new file mode 100644 index 0000000000000..2f3c1a182814d --- /dev/null +++ b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.scalastyle + +import java.util.regex.Pattern + +import org.scalastyle.{PositionError, ScalariformChecker, ScalastyleError} +import scalariform.lexer.{MultiLineComment, ScalaDocComment, SingleLineComment, Token} +import scalariform.parser.CompilationUnit + +class SparkSpaceAfterCommentStartChecker extends ScalariformChecker { + val errorKey: String = "insert.a.single.space.after.comment.start" + + private def multiLineCommentRegex(comment: Token) = + Pattern.compile( """/\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + + private def scalaDocPatternRegex(comment: Token) = + Pattern.compile( """/\*\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + + private def singleLineCommentRegex(comment: Token): Boolean = + comment.text.trim.matches( """//\S+.*""") && !comment.text.trim.matches( """///+""") + + override def verify(ast: CompilationUnit): List[ScalastyleError] = { + ast.tokens + .filter(hasComment) + .map { + _.associatedWhitespaceAndComments.comments.map { + case x: SingleLineComment if singleLineCommentRegex(x.token) => Some(x.token.offset) + case x: MultiLineComment if multiLineCommentRegex(x.token) => Some(x.token.offset) + case x: ScalaDocComment if scalaDocPatternRegex(x.token) => Some(x.token.offset) + case _ => None + }.flatten + }.flatten.map(PositionError(_)) + } + + + private def hasComment(x: Token) = + x.associatedWhitespaceAndComments != null && !x.associatedWhitespaceAndComments.comments.isEmpty + +} diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index ee972887feda6..bf73800388ebf 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -124,8 +124,8 @@ extends ClassVisitor(ASM4, cv) { mv.visitVarInsn(ALOAD, 0) // load this mv.visitMethodInsn(INVOKESPECIAL, "java/lang/Object", "", "()V") mv.visitVarInsn(ALOAD, 0) // load this - //val classType = className.replace('.', '/') - //mv.visitFieldInsn(PUTSTATIC, classType, "MODULE$", "L" + classType + ";") + // val classType = className.replace('.', '/') + // mv.visitFieldInsn(PUTSTATIC, classType, "MODULE$", "L" + classType + ";") mv.visitInsn(RETURN) mv.visitMaxs(-1, -1) // stack size and local vars will be auto-computed mv.visitEnd() diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 90a96ad38381e..fa2f1a88c4eb5 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -834,7 +834,7 @@ import org.apache.spark.util.Utils } ((pos, msg)) :: loop(filtered) } - //PRASHANT: This leads to a NoSuchMethodError for _.warnings. Yet to figure out its purpose. + // PRASHANT: This leads to a NoSuchMethodError for _.warnings. Yet to figure out its purpose. // val warnings = loop(run.allConditionalWarnings flatMap (_.warnings)) // if (warnings.nonEmpty) // mostRecentWarnings = warnings diff --git a/scalastyle-config.xml b/scalastyle-config.xml index ee968c53b3e4b..76ba1ecca33ab 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -140,4 +140,5 @@ + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index f4b61381f9a27..b70ec897e43e7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -662,7 +662,7 @@ object HiveQl { // worth the number of hacks that will be required to implement it. Namely, we need to add // some sort of mapped star expansion that would expand all child output row to be similarly // named output expressions where some aggregate expression has been applied (i.e. First). - ??? /// Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult) + ??? // Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult) case Token(allJoinTokens(joinToken), relation1 :: diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index ca5311344615f..0da5eb754cb3f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -94,7 +94,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon val tablePath = hiveTable.getPath val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) - //logDebug("Table input: %s".format(tablePath)) + // logDebug("Table input: %s".format(tablePath)) val ifc = hiveTable.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index fde46705d89fb..d3339063cc079 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -153,7 +153,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def validate() { this.synchronized { assert(batchDuration != null, "Batch duration has not been set") - //assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + + // assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + // " is very low") assert(getOutputStreams().size > 0, "No output streams registered, so nothing to execute") } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 0dc6704603f82..72ad0bae75bfb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -128,7 +128,6 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging } catch { case ie: InterruptedException => logInfo("Receiving thread interrupted") - //println("Receiving thread interrupted") case e: Exception => stopOnError(e) } @@ -142,7 +141,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging def stop() { receivingThread.interrupt() onStop() - //TODO: terminate the actor + // TODO: terminate the actor } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index ca0a8ae47864d..b334d68bf9910 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -78,7 +78,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( override def checkpoint(interval: Duration): DStream[(K, V)] = { super.checkpoint(interval) - //reducedStream.checkpoint(interval) + // reducedStream.checkpoint(interval) this } @@ -128,7 +128,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( // Cogroup the reduced RDDs and merge the reduced values val cogroupedRDD = new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(K, _)]]], partitioner) - //val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _ + // val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _ val numOldValues = oldRDDs.size val numNewValues = newRDDs.size diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 9d8889b655356..5f7d3ba26c656 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -64,7 +64,6 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( } val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner) val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning) - //logDebug("Generating state RDD for time " + validTime) Some(stateRDD) } case None => { // If parent RDD does not exist @@ -97,11 +96,11 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( val groupedRDD = parentRDD.groupByKey(partitioner) val sessionRDD = groupedRDD.mapPartitions(finalFunc, preservePartitioning) - //logDebug("Generating state RDD for time " + validTime + " (first)") + // logDebug("Generating state RDD for time " + validTime + " (first)") Some(sessionRDD) } case None => { // If parent RDD does not exist, then nothing to do! - //logDebug("Not generating state RDD (no previous state, no parent)") + // logDebug("Not generating state RDD (no previous state, no parent)") None } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index e4fa163f2e069..cad68e248ab29 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -126,7 +126,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receiverInfo -= streamId logError("De-registered receiver for network stream " + streamId + " with message " + msg) - //TODO: Do something about the corresponding NetworkInputDStream + // TODO: Do something about the corresponding NetworkInputDStream } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 0784e562ac719..25739956cb889 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -252,7 +252,7 @@ class CheckpointSuite extends TestSuiteBase { ssc.start() // Create files and advance manual clock to process them - //var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + // var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] Thread.sleep(1000) for (i <- Seq(1, 2, 3)) { Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 74e73ebb342fe..7df206241beb6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -154,7 +154,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", - StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope + // Had to pass the local value of port to prevent from closing over entire scope + StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 57e5761cba896..6568003bf1008 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -139,7 +139,6 @@ trait ClientBase extends Logging { } else if (srcHost != null && dstHost == null) { return false } - //check for ports if (srcUri.getPort() != dstUri.getPort()) { false } else { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 68cda0f1c9f8b..9b7f1fca96c6d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -157,7 +157,7 @@ class ClientDistributedCacheManager() extends Logging { def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { val fs = FileSystem.get(uri, conf) val current = new Path(uri.getPath()) - //the leaf level file should be readable by others + // the leaf level file should be readable by others if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) { return false } @@ -177,7 +177,7 @@ class ClientDistributedCacheManager() extends Logging { statCache: Map[URI, FileStatus]): Boolean = { var current = path while (current != null) { - //the subdirs in the path should have execute permissions for others + // the subdirs in the path should have execute permissions for others if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) { return false } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala index 458df4fa3cd99..80b57d1355a3a 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala @@ -99,7 +99,7 @@ class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar { assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) - //add another one and verify both there and order correct + // add another one and verify both there and order correct val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", null, new Path("/tmp/testing2")) val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2") From 75d46be5d61fb92a6db2efb9e3a690716ef521d3 Mon Sep 17 00:00:00 2001 From: Nick Lanham Date: Fri, 28 Mar 2014 13:33:35 -0700 Subject: [PATCH 18/27] fix path for jar, make sed actually work on OSX Author: Nick Lanham Closes #264 from nicklan/make-distribution-fixes and squashes the following commits: 172b981 [Nick Lanham] fix path for jar, make sed actually work on OSX --- make-distribution.sh | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index b07aef2ef49c2..5c780fcbda863 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -141,12 +141,11 @@ if [ "$SPARK_TACHYON" == "true" ]; then cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" if [[ `uname -a` == Darwin* ]]; then - # osx sed wants an empty argument to -i option of sed - TACHYON_SED="sed -i ''" + # need to run sed differently on osx + nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" else - TACHYON_SED="sed -i" + sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" fi - $TACHYON_SED -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" popd > /dev/null rm -rf $TMPD From 3738f24421d6f3bd10e5ef9ebfc10f702a5cb7ac Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 28 Mar 2014 23:09:29 -0700 Subject: [PATCH 19/27] SPARK-1345 adding missing dependency on avro for hadoop 0.23 to the new ... ...sql pom files Author: Thomas Graves Closes #263 from tgravescs/SPARK-1345 and squashes the following commits: b43a2a0 [Thomas Graves] SPARK-1345 adding missing dependency on avro for hadoop 0.23 to the new sql pom files --- sql/catalyst/pom.xml | 12 ++++++++++++ sql/core/pom.xml | 11 +++++++++++ sql/hive/pom.xml | 11 +++++++++++ 3 files changed, 34 insertions(+) diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 740f1fdc83299..0edce55a93338 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -31,6 +31,18 @@ Spark Project Catalyst http://spark.apache.org/ + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/sql/core/pom.xml b/sql/core/pom.xml index e367edfb1f562..85580ed6b822f 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -30,6 +30,17 @@ jar Spark Project SQL http://spark.apache.org/ + + + yarn-alpha + + + org.apache.avro + avro + + + + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 7b5ea98f27ff5..63f592cb4b441 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -30,6 +30,17 @@ jar Spark Project Hive http://spark.apache.org/ + + + yarn-alpha + + + org.apache.avro + avro + + + + From 1617816090e7b20124a512a43860a21232ebf511 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sat, 29 Mar 2014 14:41:36 -0700 Subject: [PATCH 20/27] SPARK-1126. spark-app preliminary This is a starting version of the spark-app script for running compiled binaries against Spark. It still needs tests and some polish. The only testing I've done so far has been using it to launch jobs in yarn-standalone mode against a pseudo-distributed cluster. This leaves out the changes required for launching python scripts. I think it might be best to save those for another JIRA/PR (while keeping to the design so that they won't require backwards-incompatible changes). Author: Sandy Ryza Closes #86 from sryza/sandy-spark-1126 and squashes the following commits: d428d85 [Sandy Ryza] Commenting, doc, and import fixes from Patrick's comments e7315c6 [Sandy Ryza] Fix failing tests 34de899 [Sandy Ryza] Change --more-jars to --jars and fix docs 299ddca [Sandy Ryza] Fix scalastyle a94c627 [Sandy Ryza] Add newline at end of SparkSubmit 04bc4e2 [Sandy Ryza] SPARK-1126. spark-submit script --- bin/spark-submit | 38 ++++ .../org/apache/spark/deploy/SparkSubmit.scala | 212 ++++++++++++++++++ .../spark/deploy/SparkSubmitArguments.scala | 176 +++++++++++++++ .../spark/deploy/SparkSubmitSuite.scala | 121 ++++++++++ docs/cluster-overview.md | 50 +++++ docs/running-on-yarn.md | 6 +- docs/spark-standalone.md | 7 +- .../cluster/YarnClientSchedulerBackend.scala | 45 ++-- 8 files changed, 630 insertions(+), 25 deletions(-) create mode 100755 bin/spark-submit create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala diff --git a/bin/spark-submit b/bin/spark-submit new file mode 100755 index 0000000000000..d92d55a032bd5 --- /dev/null +++ b/bin/spark-submit @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +# +# 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. +# + +export SPARK_HOME="$(cd `dirname $0`/..; pwd)" +ORIG_ARGS=$@ + +while (($#)); do + if [ $1 = "--deploy-mode" ]; then + DEPLOY_MODE=$2 + elif [ $1 = "--driver-memory" ]; then + DRIVER_MEMORY=$2 + fi + + shift +done + +if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" ]; then + export SPARK_MEM=$DRIVER_MEMORY +fi + +$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit $ORIG_ARGS + diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala new file mode 100644 index 0000000000000..24a9c98e188f6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.io.File +import java.net.URL + +import org.apache.spark.executor.ExecutorURLClassLoader + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + +/** + * Scala code behind the spark-submit script. The script handles setting up the classpath with + * relevant Spark dependencies and provides a layer over the different cluster managers and deploy + * modes that Spark supports. + */ +object SparkSubmit { + val YARN = 1 + val STANDALONE = 2 + val MESOS = 4 + val LOCAL = 8 + val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + + var clusterManager: Int = LOCAL + + def main(args: Array[String]) { + val appArgs = new SparkSubmitArguments(args) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + launch(childArgs, classpath, sysProps, mainClass) + } + + /** + * @return + * a tuple containing the arguments for the child, a list of classpath + * entries for the child, and the main class for the child + */ + def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], + ArrayBuffer[String], Map[String, String], String) = { + if (appArgs.master.startsWith("yarn")) { + clusterManager = YARN + } else if (appArgs.master.startsWith("spark")) { + clusterManager = STANDALONE + } else if (appArgs.master.startsWith("mesos")) { + clusterManager = MESOS + } else if (appArgs.master.startsWith("local")) { + clusterManager = LOCAL + } else { + System.err.println("master must start with yarn, mesos, spark, or local") + System.exit(1) + } + + // Because "yarn-standalone" and "yarn-client" encapsulate both the master + // and deploy mode, we have some logic to infer the master and deploy mode + // from each other if only one is specified, or exit early if they are at odds. + if (appArgs.deployMode == null && appArgs.master == "yarn-standalone") { + appArgs.deployMode = "cluster" + } + if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { + System.err.println("Deploy mode \"cluster\" and master \"yarn-client\" are at odds") + System.exit(1) + } + if (appArgs.deployMode == "client" && appArgs.master == "yarn-standalone") { + System.err.println("Deploy mode \"client\" and master \"yarn-standalone\" are at odds") + System.exit(1) + } + if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { + appArgs.master = "yarn-standalone" + } + if (appArgs.deployMode != "cluster" && appArgs.master.startsWith("yarn")) { + appArgs.master = "yarn-client" + } + + val deployOnCluster = Option(appArgs.deployMode).getOrElse("client") == "cluster" + + val childClasspath = new ArrayBuffer[String]() + val childArgs = new ArrayBuffer[String]() + val sysProps = new HashMap[String, String]() + var childMainClass = "" + + if (clusterManager == MESOS && deployOnCluster) { + System.err.println("Mesos does not support running the driver on the cluster") + System.exit(1) + } + + if (!deployOnCluster) { + childMainClass = appArgs.mainClass + childClasspath += appArgs.primaryResource + } else if (clusterManager == YARN) { + childMainClass = "org.apache.spark.deploy.yarn.Client" + childArgs += ("--jar", appArgs.primaryResource) + childArgs += ("--class", appArgs.mainClass) + } + + val options = List[OptionAssigner]( + new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), + new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), + new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), + new OptionAssigner(appArgs.queue, YARN, false, sysProp = "spark.yarn.queue"), + new OptionAssigner(appArgs.numExecutors, YARN, true, clOption = "--num-executors"), + new OptionAssigner(appArgs.numExecutors, YARN, false, sysProp = "spark.executor.instances"), + new OptionAssigner(appArgs.executorMemory, YARN, true, clOption = "--executor-memory"), + new OptionAssigner(appArgs.executorMemory, STANDALONE | MESOS | YARN, false, + sysProp = "spark.executor.memory"), + new OptionAssigner(appArgs.driverMemory, STANDALONE, true, clOption = "--memory"), + new OptionAssigner(appArgs.driverCores, STANDALONE, true, clOption = "--cores"), + new OptionAssigner(appArgs.executorCores, YARN, true, clOption = "--executor-cores"), + new OptionAssigner(appArgs.executorCores, YARN, false, sysProp = "spark.executor.cores"), + new OptionAssigner(appArgs.totalExecutorCores, STANDALONE | MESOS, false, + sysProp = "spark.cores.max"), + new OptionAssigner(appArgs.files, YARN, false, sysProp = "spark.yarn.dist.files"), + new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), + new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), + new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), + new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars") + ) + + // more jars + if (appArgs.jars != null && !deployOnCluster) { + for (jar <- appArgs.jars.split(",")) { + childClasspath += jar + } + } + + for (opt <- options) { + if (opt.value != null && deployOnCluster == opt.deployOnCluster && + (clusterManager & opt.clusterManager) != 0) { + if (opt.clOption != null) { + childArgs += (opt.clOption, opt.value) + } else if (opt.sysProp != null) { + sysProps.put(opt.sysProp, opt.value) + } + } + } + + if (deployOnCluster && clusterManager == STANDALONE) { + if (appArgs.supervise) { + childArgs += "--supervise" + } + + childMainClass = "org.apache.spark.deploy.Client" + childArgs += "launch" + childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) + } + + // args + if (appArgs.childArgs != null) { + if (!deployOnCluster || clusterManager == STANDALONE) { + childArgs ++= appArgs.childArgs + } else if (clusterManager == YARN) { + for (arg <- appArgs.childArgs) { + childArgs += ("--args", arg) + } + } + } + + (childArgs, childClasspath, sysProps, childMainClass) + } + + def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], + sysProps: Map[String, String], childMainClass: String) { + val loader = new ExecutorURLClassLoader(new Array[URL](0), + Thread.currentThread.getContextClassLoader) + Thread.currentThread.setContextClassLoader(loader) + + for (jar <- childClasspath) { + addJarToClasspath(jar, loader) + } + + for ((key, value) <- sysProps) { + System.setProperty(key, value) + } + + val mainClass = Class.forName(childMainClass, true, loader) + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) + mainMethod.invoke(null, childArgs.toArray) + } + + def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { + val localJarFile = new File(localJar) + if (!localJarFile.exists()) { + System.err.println("Jar does not exist: " + localJar + ". Skipping.") + } + + val url = localJarFile.getAbsoluteFile.toURI.toURL + loader.addURL(url) + } +} + +private[spark] class OptionAssigner(val value: String, + val clusterManager: Int, + val deployOnCluster: Boolean, + val clOption: String = null, + val sysProp: String = null +) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala new file mode 100644 index 0000000000000..ff2aa68908e34 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import scala.collection.mutable.ArrayBuffer + +/** + * Parses and encapsulates arguments from the spark-submit script. + */ +private[spark] class SparkSubmitArguments(args: Array[String]) { + var master: String = "local" + var deployMode: String = null + var executorMemory: String = null + var executorCores: String = null + var totalExecutorCores: String = null + var driverMemory: String = null + var driverCores: String = null + var supervise: Boolean = false + var queue: String = null + var numExecutors: String = null + var files: String = null + var archives: String = null + var mainClass: String = null + var primaryResource: String = null + var name: String = null + var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]() + var jars: String = null + + loadEnvVars() + parseArgs(args.toList) + + def loadEnvVars() { + master = System.getenv("MASTER") + deployMode = System.getenv("DEPLOY_MODE") + } + + def parseArgs(args: List[String]) { + if (args.size == 0) { + printUsageAndExit(1) + System.exit(1) + } + primaryResource = args(0) + parseOpts(args.tail) + } + + def parseOpts(opts: List[String]): Unit = opts match { + case ("--name") :: value :: tail => + name = value + parseOpts(tail) + + case ("--master") :: value :: tail => + master = value + parseOpts(tail) + + case ("--class") :: value :: tail => + mainClass = value + parseOpts(tail) + + case ("--deploy-mode") :: value :: tail => + if (value != "client" && value != "cluster") { + System.err.println("--deploy-mode must be either \"client\" or \"cluster\"") + System.exit(1) + } + deployMode = value + parseOpts(tail) + + case ("--num-executors") :: value :: tail => + numExecutors = value + parseOpts(tail) + + case ("--total-executor-cores") :: value :: tail => + totalExecutorCores = value + parseOpts(tail) + + case ("--executor-cores") :: value :: tail => + executorCores = value + parseOpts(tail) + + case ("--executor-memory") :: value :: tail => + executorMemory = value + parseOpts(tail) + + case ("--driver-memory") :: value :: tail => + driverMemory = value + parseOpts(tail) + + case ("--driver-cores") :: value :: tail => + driverCores = value + parseOpts(tail) + + case ("--supervise") :: tail => + supervise = true + parseOpts(tail) + + case ("--queue") :: value :: tail => + queue = value + parseOpts(tail) + + case ("--files") :: value :: tail => + files = value + parseOpts(tail) + + case ("--archives") :: value :: tail => + archives = value + parseOpts(tail) + + case ("--arg") :: value :: tail => + childArgs += value + parseOpts(tail) + + case ("--jars") :: value :: tail => + jars = value + parseOpts(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => + + case _ => + printUsageAndExit(1, opts) + } + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + if (unknownParam != null) { + System.err.println("Unknown/unsupported param " + unknownParam) + } + System.err.println( + """Usage: spark-submit [options] + |Options: + | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. + | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. + | --class CLASS_NAME Name of your app's main class (required for Java apps). + | --arg ARG Argument to be passed to your application's main class. This + | option can be specified multiple times for multiple args. + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). + | --name NAME The name of your application (Default: 'Spark'). + | --jars JARS A comma-separated list of local jars to include on the + | driver classpath and that SparkContext.addJar will work + | with. Doesn't work on standalone with 'cluster' deploy mode. + | + | Spark standalone with cluster deploy mode only: + | --driver-cores NUM Cores for driver (Default: 1). + | --supervise If given, restarts the driver on failure. + | + | Spark standalone and Mesos only: + | --total-executor-cores NUM Total cores for all executors. + | + | YARN-only: + | --executor-cores NUM Number of cores per executor (Default: 1). + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). + | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). + | --num-executors NUM Number of executors to (Default: 2). + | --files FILES Comma separated list of files to be placed in the working dir + | of each executor. + | --archives ARCHIVES Comma separated list of archives to be extracted into the + | working dir of each executor.""".stripMargin + ) + System.exit(exitCode) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala new file mode 100644 index 0000000000000..29fef2ed8c165 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers +import org.apache.spark.deploy.SparkSubmit._ + +class SparkSubmitSuite extends FunSuite with ShouldMatchers { + test("prints usage on empty input") { + val clArgs = Array[String]() + // val appArgs = new SparkSubmitArguments(clArgs) + } + + test("handles YARN cluster mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", + "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", + "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", + "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", + "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val childArgsStr = childArgs.mkString(" ") + childArgsStr should include ("--jar thejar.jar") + childArgsStr should include ("--class org.SomeClass") + childArgsStr should include ("--addJars one.jar,two.jar,three.jar") + childArgsStr should include ("--executor-memory 5g") + childArgsStr should include ("--driver-memory 4g") + childArgsStr should include ("--executor-cores 5") + childArgsStr should include ("--args arg1 --args arg2") + childArgsStr should include ("--queue thequeue") + childArgsStr should include ("--files file1.txt,file2.txt") + childArgsStr should include ("--archives archive1.txt,archive2.txt") + childArgsStr should include ("--num-executors 6") + mainClass should be ("org.apache.spark.deploy.yarn.Client") + classpath should have length (0) + sysProps should have size (0) + } + + test("handles YARN client mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "client", + "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", + "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", + "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", + "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + childArgs.mkString(" ") should be ("arg1 arg2") + mainClass should be ("org.SomeClass") + classpath should contain ("thejar.jar") + classpath should contain ("one.jar") + classpath should contain ("two.jar") + classpath should contain ("three.jar") + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.executor.cores") should be ("5") + sysProps("spark.yarn.queue") should be ("thequeue") + sysProps("spark.yarn.dist.files") should be ("file1.txt,file2.txt") + sysProps("spark.yarn.dist.archives") should be ("archive1.txt,archive2.txt") + sysProps("spark.executor.instances") should be ("6") + } + + test("handles standalone cluster mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", + "--master", "spark://h:p", "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", + "--supervise", "--driver-memory", "4g", "--driver-cores", "5") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val childArgsStr = childArgs.mkString(" ") + print("child args: " + childArgsStr) + childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) + childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") + mainClass should be ("org.apache.spark.deploy.Client") + classpath should have length (0) + sysProps should have size (0) + } + + test("handles standalone client mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "client", + "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", + "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", + "--driver-memory", "4g") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + childArgs.mkString(" ") should be ("arg1 arg2") + mainClass should be ("org.SomeClass") + classpath should contain ("thejar.jar") + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.cores.max") should be ("5") + } + + test("handles mesos client mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "client", + "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", + "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", + "--driver-memory", "4g") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + childArgs.mkString(" ") should be ("arg1 arg2") + mainClass should be ("org.SomeClass") + classpath should contain ("thejar.jar") + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.cores.max") should be ("5") + } +} diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index a555a7b5023e3..b69e3416fb322 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -50,6 +50,50 @@ The system currently supports three cluster managers: In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone cluster on Amazon EC2. +# Launching Applications + +The recommended way to launch a compiled Spark application is through the spark-submit script (located in the +bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as +provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is + + spark-submit `` `` + +Where options are any of: + +- **\--class** - The main class to run. +- **\--master** - The URL of the cluster manager master, e.g. spark://host:port, mesos://host:port, yarn, + or local. +- **\--deploy-mode** - "client" to run the driver in the client process or "cluster" to run the driver in + a process on the cluster. For Mesos, only "client" is supported. +- **\--executor-memory** - Memory per executor (e.g. 1000M, 2G). +- **\--executor-cores** - Number of cores per executor. (Default: 2) +- **\--driver-memory** - Memory for driver (e.g. 1000M, 2G) +- **\--name** - Name of the application. +- **\--arg** - Argument to be passed to the application's main class. This option can be specified + multiple times to pass multiple arguments. +- **\--jars** - A comma-separated list of local jars to include on the driver classpath and that + SparkContext.addJar will work with. Doesn't work on standalone with 'cluster' deploy mode. + +The following currently only work for Spark standalone with cluster deploy mode: + +- **\--driver-cores** - Cores for driver (Default: 1). +- **\--supervise** - If given, restarts the driver on failure. + +The following only works for Spark standalone and Mesos only: + +- **\--total-executor-cores** - Total cores for all executors. + +The following currently only work for YARN: + +- **\--queue** - The YARN queue to place the application in. +- **\--files** - Comma separated list of files to be placed in the working dir of each executor. +- **\--archives** - Comma separated list of archives to be extracted into the working dir of each + executor. +- **\--num-executors** - Number of executors (Default: 2). + +The master and deploy mode can also be set with the MASTER and DEPLOY_MODE environment variables. +Values for these options passed via command line will override the environment variables. + # Shipping Code to the Cluster The recommended way to ship your code to the cluster is to pass it through SparkContext's constructor, @@ -102,6 +146,12 @@ The following table summarizes terms you'll see used to refer to cluster concept Cluster manager An external service for acquiring resources on the cluster (e.g. standalone manager, Mesos, YARN) + + Deploy mode + Distinguishes where the driver process runs. In "cluster" mode, the framework launches + the driver inside of the cluster. In "client" mode, the submitter launches the driver + outside of the cluster. + Worker node Any node that can run application code in the cluster diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 2e9dec4856ee9..d8657c4bc7096 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,10 +48,12 @@ System Properties: Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the Hadoop cluster. These configs are used to connect to the cluster, write to the dfs, and connect to the YARN ResourceManager. -There are two scheduler modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. +There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. Unlike in Spark standalone and Mesos mode, in which the master's address is specified in the "master" parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the master parameter is simply "yarn-client" or "yarn-cluster". +The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to YARN in either deploy mode. For info on the lower-level invocations it uses, read ahead. For running spark-shell against YARN, skip down to the yarn-client section. + ## Launching a Spark application with yarn-cluster mode. The command to launch the Spark application on the cluster is as follows: @@ -121,7 +123,7 @@ or MASTER=yarn-client ./bin/spark-shell -## Viewing logs +# Viewing logs In YARN terminology, executors and application masters run inside "containers". YARN has two modes for handling container logs after an application has completed. If log aggregation is turned on (with the yarn.log-aggregation-enable config), container logs are copied to HDFS and deleted on the local machine. These logs can be viewed from anywhere on the cluster with the "yarn logs" command. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 51fb3a4f7f8c5..7e4eea323aa63 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -146,10 +146,13 @@ automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` vari You can also pass an option `-c ` to control the number of cores that spark-shell uses on the cluster. -# Launching Applications Inside the Cluster +# Launching Compiled Spark Applications -You may also run your application entirely inside of the cluster by submitting your application driver using the submission client. The syntax for submitting applications is as follows: +Spark supports two deploy modes. Spark applications may run with the driver inside the client process or entirely inside the cluster. +The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to the cluster in either deploy mode. For info on the lower-level invocations used to launch an app inside the cluster, read ahead. + +## Launching Applications Inside the Cluster ./bin/spark-class org.apache.spark.deploy.Client launch [client-options] \ diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index d1f13e3c369ed..161918859e7c4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -33,11 +33,12 @@ private[spark] class YarnClientSchedulerBackend( var client: Client = null var appId: ApplicationId = null - private[spark] def addArg(optionName: String, optionalParam: String, arrayBuf: ArrayBuffer[String]) { - Option(System.getenv(optionalParam)) foreach { - optParam => { - arrayBuf += (optionName, optParam) - } + private[spark] def addArg(optionName: String, envVar: String, sysProp: String, + arrayBuf: ArrayBuffer[String]) { + if (System.getProperty(sysProp) != null) { + arrayBuf += (optionName, System.getProperty(sysProp)) + } else if (System.getenv(envVar) != null) { + arrayBuf += (optionName, System.getenv(envVar)) } } @@ -56,22 +57,24 @@ private[spark] class YarnClientSchedulerBackend( "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" ) - // process any optional arguments, use the defaults already defined in ClientArguments - // if things aren't specified - Map("SPARK_MASTER_MEMORY" -> "--driver-memory", - "SPARK_DRIVER_MEMORY" -> "--driver-memory", - "SPARK_WORKER_INSTANCES" -> "--num-executors", - "SPARK_WORKER_MEMORY" -> "--executor-memory", - "SPARK_WORKER_CORES" -> "--executor-cores", - "SPARK_EXECUTOR_INSTANCES" -> "--num-executors", - "SPARK_EXECUTOR_MEMORY" -> "--executor-memory", - "SPARK_EXECUTOR_CORES" -> "--executor-cores", - "SPARK_YARN_QUEUE" -> "--queue", - "SPARK_YARN_APP_NAME" -> "--name", - "SPARK_YARN_DIST_FILES" -> "--files", - "SPARK_YARN_DIST_ARCHIVES" -> "--archives") - .foreach { case (optParam, optName) => addArg(optName, optParam, argsArrayBuf) } - + // process any optional arguments, given either as environment variables + // or system properties. use the defaults already defined in ClientArguments + // if things aren't specified. system properties override environment + // variables. + List(("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), + ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), + ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.worker.instances"), + ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), + ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), + ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), + ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), + ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), + ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), + ("--name", "SPARK_YARN_APP_NAME", "spark.app.name"), + ("--files", "SPARK_YARN_DIST_FILES", "spark.yarn.dist.files"), + ("--archives", "SPARK_YARN_DIST_ARCHIVES", "spark.yarn.dist.archives")) + .foreach { case (optName, envVar, sysProp) => addArg(optName, envVar, sysProp, argsArrayBuf) } + logDebug("ClientArguments called with: " + argsArrayBuf) val args = new ClientArguments(argsArrayBuf.toArray, conf) client = new Client(args, conf) From af3746ce0d724dc624658a2187bde188ab26d084 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sat, 29 Mar 2014 15:12:43 -0700 Subject: [PATCH 21/27] Implement the RLike & Like in catalyst This PR includes: 1) Unify the unit test for expression evaluation 2) Add implementation of RLike & Like Author: Cheng Hao Closes #224 from chenghao-intel/string_expression and squashes the following commits: 84f72e9 [Cheng Hao] fix bug in RLike/Like & Simplify the unit test aeeb1d7 [Cheng Hao] Simplify the implementation/unit test of RLike/Like 319edb7 [Cheng Hao] change to spark code style 91cfd33 [Cheng Hao] add implementation for rlike/like 2c8929e [Cheng Hao] Update the unit test for expression evaluation --- .../apache/spark/sql/catalyst/SqlParser.scala | 6 ++ .../spark/sql/catalyst/dsl/package.scala | 10 +- .../expressions/stringOperations.scala | 98 ++++++++++++++++++- .../ExpressionEvaluationSuite.scala | 83 ++++++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 9 +- 5 files changed, 196 insertions(+), 10 deletions(-) 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 9dec4e3d9e4c2..0c851c2ee2183 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 @@ -114,6 +114,9 @@ class SqlParser extends StandardTokenParsers { protected val NULL = Keyword("NULL") protected val ON = Keyword("ON") protected val OR = Keyword("OR") + protected val LIKE = Keyword("LIKE") + protected val RLIKE = Keyword("RLIKE") + protected val REGEXP = Keyword("REGEXP") protected val ORDER = Keyword("ORDER") protected val OUTER = Keyword("OUTER") protected val RIGHT = Keyword("RIGHT") @@ -267,6 +270,9 @@ class SqlParser extends StandardTokenParsers { termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } | termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } | + termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | + termExpression ~ REGEXP ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | + termExpression ~ LIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => Like(e1, e2) } | termExpression ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ { case e1 ~ _ ~ _ ~ e2 => In(e1, e2) } | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 67cddb351c185..44abe671c07a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -70,6 +70,9 @@ package object dsl { def === (other: Expression) = Equals(expr, other) def != (other: Expression) = Not(Equals(expr, other)) + def like(other: Expression) = Like(expr, other) + def rlike(other: Expression) = RLike(expr, other) + def asc = SortOrder(expr, Ascending) def desc = SortOrder(expr, Descending) @@ -90,7 +93,10 @@ package object dsl { implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name) implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } - implicit class DslString(val s: String) extends ImplicitAttribute + implicit class DslString(val s: String) extends ImplicitOperators { + def expr: Expression = Literal(s) + def attr = analysis.UnresolvedAttribute(s) + } abstract class ImplicitAttribute extends ImplicitOperators { def s: String @@ -110,6 +116,8 @@ package object dsl { // Protobuf terminology def required = a.withNullability(false) + + def at(ordinal: Int) = BoundReference(ordinal, a) } } 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 e195f2ac7efd1..42b7a9b125b7a 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 @@ -17,11 +17,103 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.regex.Pattern + +import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.catalyst.types.BooleanType +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.errors.`package`.TreeNodeException + + +trait StringRegexExpression { + self: BinaryExpression => + + type EvaluatedType = Any + + def escape(v: String): String + def matches(regex: Pattern, str: String): Boolean + + def nullable: Boolean = true + def dataType: DataType = BooleanType + + // try cache the pattern for Literal + private lazy val cache: Pattern = right match { + case x @ Literal(value: String, StringType) => compile(value) + case _ => null + } + + protected def compile(str: String): Pattern = if(str == null) { + null + } else { + // Let it raise exception if couldn't compile the regex string + Pattern.compile(escape(str)) + } -case class Like(left: Expression, right: Expression) extends BinaryExpression { - def dataType = BooleanType - def nullable = left.nullable // Right cannot be null. + protected def pattern(str: String) = if(cache == null) compile(str) else cache + + override def apply(input: Row): Any = { + val l = left.apply(input) + if(l == null) { + null + } else { + val r = right.apply(input) + if(r == null) { + null + } else { + val regex = pattern(r.asInstanceOf[String]) + if(regex == null) { + null + } else { + matches(regex, l.asInstanceOf[String]) + } + } + } + } +} + +/** + * Simple RegEx pattern matching function + */ +case class Like(left: Expression, right: Expression) + extends BinaryExpression with StringRegexExpression { + def symbol = "LIKE" + + // replace the _ with .{1} exactly match 1 time of any character + // replace the % with .*, match 0 or more times with any character + override def escape(v: String) = { + val sb = new StringBuilder() + var i = 0; + while (i < v.length) { + // Make a special case for "\\_" and "\\%" + val n = v.charAt(i); + if (n == '\\' && i + 1 < v.length && (v.charAt(i + 1) == '_' || v.charAt(i + 1) == '%')) { + sb.append(v.charAt(i + 1)) + i += 1 + } else { + if (n == '_') { + sb.append("."); + } else if (n == '%') { + sb.append(".*"); + } else { + sb.append(Pattern.quote(Character.toString(n))); + } + } + + i += 1 + } + + sb.toString() + } + + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() } +case class RLike(left: Expression, right: Expression) + extends BinaryExpression with StringRegexExpression { + + def symbol = "RLIKE" + override def escape(v: String): String = v + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 94894adf81202..52a205be3e9f4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -109,4 +109,87 @@ class ExpressionEvaluationSuite extends FunSuite { } } } + + def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { + expression.apply(inputRow) + } + + def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + if(actual != expected) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + test("LIKE literal Regular Expression") { + checkEvaluation(Literal(null, StringType).like("a"), null) + checkEvaluation(Literal(null, StringType).like(Literal(null, StringType)), null) + checkEvaluation("abdef" like "abdef", true) + checkEvaluation("a_%b" like "a\\__b", true) + checkEvaluation("addb" like "a_%b", true) + checkEvaluation("addb" like "a\\__b", false) + checkEvaluation("addb" like "a%\\%b", false) + checkEvaluation("a_%b" like "a%\\%b", true) + checkEvaluation("addb" like "a%", true) + checkEvaluation("addb" like "**", false) + checkEvaluation("abc" like "a%", true) + checkEvaluation("abc" like "b%", false) + checkEvaluation("abc" like "bc%", false) + } + + test("LIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) + checkEvaluation("abdef" like regEx, true, new GenericRow(Array[Any]("abdef"))) + checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a\\__b"))) + checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a_%b"))) + checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a\\__b"))) + checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a%\\%b"))) + checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a%\\%b"))) + checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a%"))) + checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("**"))) + checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) + checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) + checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) + } + + test("RLIKE literal Regular Expression") { + checkEvaluation("abdef" rlike "abdef", true) + checkEvaluation("abbbbc" rlike "a.*c", true) + + checkEvaluation("fofo" rlike "^fo", true) + checkEvaluation("fo\no" rlike "^fo\no$", true) + checkEvaluation("Bn" rlike "^Ba*n", true) + checkEvaluation("afofo" rlike "fo", true) + checkEvaluation("afofo" rlike "^fo", false) + checkEvaluation("Baan" rlike "^Ba?n", false) + checkEvaluation("axe" rlike "pi|apa", false) + checkEvaluation("pip" rlike "^(pi)*$", false) + + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + + intercept[java.util.regex.PatternSyntaxException] { + evaluate("abbbbc" rlike "**") + } + } + + test("RLIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + checkEvaluation("abdef" rlike regEx, true, new GenericRow(Array[Any]("abdef"))) + checkEvaluation("abbbbc" rlike regEx, true, new GenericRow(Array[Any]("a.*c"))) + checkEvaluation("fofo" rlike regEx, true, new GenericRow(Array[Any]("^fo"))) + checkEvaluation("fo\no" rlike regEx, true, new GenericRow(Array[Any]("^fo\no$"))) + checkEvaluation("Bn" rlike regEx, true, new GenericRow(Array[Any]("^Ba*n"))) + + intercept[java.util.regex.PatternSyntaxException] { + evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) + } + } } + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index b70ec897e43e7..490a592a588d0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -847,12 +847,9 @@ object HiveQl { case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token("LIKE", left :: right:: Nil) => - UnresolvedFunction("LIKE", Seq(nodeToExpr(left), nodeToExpr(right))) - case Token("RLIKE", left :: right:: Nil) => - UnresolvedFunction("RLIKE", Seq(nodeToExpr(left), nodeToExpr(right))) - case Token("REGEXP", left :: right:: Nil) => - UnresolvedFunction("REGEXP", Seq(nodeToExpr(left), nodeToExpr(right))) + case Token("LIKE", left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) + case Token("RLIKE", left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token("REGEXP", left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => IsNotNull(nodeToExpr(child)) case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => From fda86d8b46a1cc484d11ac5446d8cc2a86429b9b Mon Sep 17 00:00:00 2001 From: Bernardo Gomez Palacio Date: Sat, 29 Mar 2014 19:49:22 -0700 Subject: [PATCH 22/27] [SPARK-1186] : Enrich the Spark Shell to support additional arguments. Enrich the Spark Shell functionality to support the following options. ``` Usage: spark-shell [OPTIONS] OPTIONS: -h --help : Print this help information. -c --cores : The maximum number of cores to be used by the Spark Shell. -em --executor-memory : The memory used by each executor of the Spark Shell, the number is followed by m for megabytes or g for gigabytes, e.g. "1g". -dm --driver-memory : The memory used by the Spark Shell, the number is followed by m for megabytes or g for gigabytes, e.g. "1g". -m --master : A full string that describes the Spark Master, defaults to "local" e.g. "spark://localhost:7077". --log-conf : Enables logging of the supplied SparkConf as INFO at start of the Spark Context. e.g. spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g ``` **Note**: this commit reflects the changes applied to _master_ based on [5d98cfc1]. [ticket: SPARK-1186] : Enrich the Spark Shell to support additional arguments. https://spark-project.atlassian.net/browse/SPARK-1186 Author : bernardo.gomezpalcio@gmail.com Author: Bernardo Gomez Palacio Closes #116 from berngp/feature/enrich-spark-shell and squashes the following commits: c5f455f [Bernardo Gomez Palacio] [SPARK-1186] : Enrich the Spark Shell to support additional arguments. --- bin/spark-shell | 226 +++++++++++++++++++++++++++++++++++------------- 1 file changed, 168 insertions(+), 58 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 861ab606540cd..fac006cf492ed 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -30,67 +30,189 @@ esac # Enter posix mode for bash set -o posix -CORE_PATTERN="^[0-9]+$" -MEM_PATTERN="^[0-9]+[m|g|M|G]$" - +## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" -if [ "$1" = "--help" ] || [ "$1" = "-h" ]; then - echo "Usage: spark-shell [OPTIONS]" - echo "OPTIONS:" - echo "-c --cores num, the maximum number of cores to be used by the spark shell" - echo "-em --execmem num[m|g], the memory used by each executor of spark shell" - echo "-dm --drivermem num[m|g], the memory used by the spark shell and driver" - echo "-h --help, print this help information" - exit -fi +SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" +DEFAULT_MASTER="local" +MASTER=${MASTER:-""} + +info_log=0 + +#CLI Color Templates +txtund=$(tput sgr 0 1) # Underline +txtbld=$(tput bold) # Bold +bldred=${txtbld}$(tput setaf 1) # red +bldyel=${txtbld}$(tput setaf 3) # yellow +bldblu=${txtbld}$(tput setaf 4) # blue +bldwht=${txtbld}$(tput setaf 7) # white +txtrst=$(tput sgr0) # Reset +info=${bldwht}*${txtrst} # Feedback +pass=${bldblu}*${txtrst} +warn=${bldred}*${txtrst} +ques=${bldblu}?${txtrst} + +# Helper function to describe the script usage +function usage() { + cat << EOF +${txtbld}Usage${txtrst}: spark-shell [OPTIONS] + +${txtbld}OPTIONS${txtrst}: + -h --help : Print this help information. + -c --cores : The maximum number of cores to be used by the Spark Shell. + -em --executor-memory : The memory used by each executor of the Spark Shell, the number + is followed by m for megabytes or g for gigabytes, e.g. "1g". + -dm --driver-memory : The memory used by the Spark Shell, the number is followed + by m for megabytes or g for gigabytes, e.g. "1g". + -m --master : A full string that describes the Spark Master, defaults to "local" + e.g. "spark://localhost:7077". + --log-conf : Enables logging of the supplied SparkConf as INFO at start of the + Spark Context. + +e.g. + spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g + +EOF +} + +function out_error(){ + echo -e "${txtund}${bldred}ERROR${txtrst}: $1" + usage + exit 1 +} + +function log_info(){ + [ $info_log -eq 1 ] && echo -e "${bldyel}INFO${txtrst}: $1" +} + +function log_warn(){ + echo -e "${txtund}${bldyel}WARN${txtrst}: $1" +} -for o in "$@"; do - if [ "$1" = "-c" -o "$1" = "--cores" ]; then - shift +# PATTERNS used to validate more than one optional arg. +ARG_FLAG_PATTERN="^-" +MEM_PATTERN="^[0-9]+[m|g|M|G]$" +NUM_PATTERN="^[0-9]+$" +PORT_PATTERN="^[0-9]+$" + +# Setters for optional args. +function set_cores(){ + CORE_PATTERN="^[0-9]+$" if [[ "$1" =~ $CORE_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" - shift + SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" else - echo "ERROR: wrong format for -c/--cores" - exit 1 + out_error "wrong format for $2" fi - fi - if [ "$1" = "-em" -o "$1" = "--execmem" ]; then - shift +} + +function set_em(){ if [[ $1 =~ $MEM_PATTERN ]]; then SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1" - shift else - echo "ERROR: wrong format for --execmem/-em" - exit 1 + out_error "wrong format for $2" fi - fi - if [ "$1" = "-dm" -o "$1" = "--drivermem" ]; then - shift +} + +function set_dm(){ if [[ $1 =~ $MEM_PATTERN ]]; then export SPARK_DRIVER_MEMORY=$1 - shift else - echo "ERROR: wrong format for --drivermem/-dm" - exit 1 + out_error "wrong format for $2" fi - fi -done +} + +function set_spark_log_conf(){ + SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.logConf=$1" +} -# Set MASTER from spark-env if possible -DEFAULT_SPARK_MASTER_PORT=7077 -if [ -z "$MASTER" ]; then - . $FWDIR/bin/load-spark-env.sh - if [ "x" != "x$SPARK_MASTER_IP" ]; then - if [ "y" != "y$SPARK_MASTER_PORT" ]; then - SPARK_MASTER_PORT="${SPARK_MASTER_PORT}" +function set_spark_master(){ + if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then + MASTER="$1" else - SPARK_MASTER_PORT=$DEFAULT_SPARK_MASTER_PORT + out_error "wrong format for $2" + fi +} + +function resolve_spark_master(){ + # Set MASTER from spark-env if possible + DEFAULT_SPARK_MASTER_PORT=7077 + if [ -z "$MASTER" ]; then + . $FWDIR/bin/load-spark-env.sh + if [ -n "$SPARK_MASTER_IP" ]; then + SPARK_MASTER_PORT="${SPARK_MASTER_PORT:-"$DEFAULT_SPARK_MASTER_PORT"}" + export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + fi + fi + + if [ -z "$MASTER" ]; then + MASTER="$DEFAULT_MASTER" fi - export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi -fi + +} + +function main(){ + log_info "Base Directory set to $FWDIR" + + resolve_spark_master + log_info "Spark Master is $MASTER" + + log_info "Spark REPL options $SPARK_REPL_OPTS" + if $cygwin; then + # Workaround for issue involving JLine and Cygwin + # (see http://sourceforge.net/p/jline/bugs/40/). + # If you're using the Mintty terminal emulator in Cygwin, may need to set the + # "Backspace sends ^H" setting in "Keys" section of the Mintty options + # (see https://github.com/sbt/sbt/issues/562). + stty -icanon min 1 -echo > /dev/null 2>&1 + export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + stty icanon echo > /dev/null 2>&1 + else + export SPARK_REPL_OPTS + $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + fi +} + +for option in "$@" +do + case $option in + -h | --help ) + usage + exit 1 + ;; + -c | --cores) + shift + _1=$1 + shift + set_cores $_1 "-c/--cores" + ;; + -em | --executor-memory) + shift + _1=$1 + shift + set_em $_1 "-em/--executor-memory" + ;; + -dm | --driver-memory) + shift + _1=$1 + shift + set_dm $_1 "-dm/--driver-memory" + ;; + -m | --master) + shift + _1=$1 + shift + set_spark_master $_1 "-m/--master" + ;; + --log-conf) + shift + set_spark_log_conf "true" + info_log=1 + ;; + ?) + ;; + esac +done # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed @@ -120,22 +242,10 @@ if [[ ! $? ]]; then saved_stty="" fi -if $cygwin; then - # Workaround for issue involving JLine and Cygwin - # (see http://sourceforge.net/p/jline/bugs/40/). - # If you're using the Mintty terminal emulator in Cygwin, may need to set the - # "Backspace sends ^H" setting in "Keys" section of the Mintty options - # (see https://github.com/sbt/sbt/issues/562). - stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" - stty icanon echo > /dev/null 2>&1 -else - export SPARK_REPL_OPTS - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" -fi +main # record the exit status lest it be overwritten: # then reenable echo and propagate the code. exit_status=$? onExit + From 92b83959cacbc902ff0b50110261f097bf2df247 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 29 Mar 2014 22:01:29 -0700 Subject: [PATCH 23/27] Don't swallow all kryo errors, only those that indicate we are out of data. Author: Michael Armbrust Closes #142 from marmbrus/kryoErrors and squashes the following commits: 9c72d1f [Michael Armbrust] Make the test more future proof. 78f5a42 [Michael Armbrust] Don't swallow all kryo errors, only those that indicate we are out of data. --- .../scala/org/apache/spark/serializer/KryoSerializer.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 6b6d814c1fe92..926e71573be32 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -107,7 +107,8 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser kryo.readClassAndObject(input).asInstanceOf[T] } catch { // DeserializationStream uses the EOF exception to indicate stopping condition. - case _: KryoException => throw new EOFException + case e: KryoException if e.getMessage.toLowerCase.contains("buffer underflow") => + throw new EOFException } } From 2861b07bb030f72769f5b757b4a7d4a635807140 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 29 Mar 2014 22:02:53 -0700 Subject: [PATCH 24/27] [SQL] SPARK-1354 Fix self-joins of parquet relations @AndreSchumacher, please take a look. https://spark-project.atlassian.net/browse/SPARK-1354 Author: Michael Armbrust Closes #269 from marmbrus/parquetJoin and squashes the following commits: 4081e77 [Michael Armbrust] Create new instances of Parquet relation when multiple copies are in a single plan. --- .../spark/sql/parquet/ParquetRelation.scala | 15 +++++++++++++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 19 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 2b825f84ee910..67a34e1f21cc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -36,7 +36,7 @@ import parquet.schema.{MessageType, MessageTypeParser} import parquet.schema.{PrimitiveType => ParquetPrimitiveType} import parquet.schema.{Type => ParquetType} -import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} import org.apache.spark.sql.catalyst.types._ @@ -54,7 +54,8 @@ import org.apache.spark.sql.catalyst.types._ * @param tableName The name of the relation that can be used in queries. * @param path The path to the Parquet file. */ -case class ParquetRelation(tableName: String, path: String) extends BaseRelation { +case class ParquetRelation(tableName: String, path: String) + extends BaseRelation with MultiInstanceRelation { /** Schema derived from ParquetFile **/ def parquetSchema: MessageType = @@ -74,6 +75,16 @@ case class ParquetRelation(tableName: String, path: String) extends BaseRelation // Parquet files have no concepts of keys, therefore no Partitioner // Note: we could allow Block level access; needs to be thought through override def isPartitioned = false + + override def newInstance = ParquetRelation(tableName, path).asInstanceOf[this.type] + + // Equals must also take into account the output attributes so that we can distinguish between + // different instances of the same relation, + override def equals(other: Any) = other match { + case p: ParquetRelation => + p.tableName == tableName && p.path == path && p.output == output + case _ => false + } } object ParquetRelation { 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 71caa709afca6..ea1733b3614e5 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 @@ -30,6 +30,9 @@ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext +// Implicits +import org.apache.spark.sql.test.TestSQLContext._ + class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { ParquetTestData.writeFile() @@ -39,6 +42,22 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { ParquetTestData.testFile.delete() } + test("self-join parquet files") { + val x = ParquetTestData.testData.subquery('x) + val y = ParquetTestData.testData.subquery('y) + val query = x.join(y).where("x.myint".attr === "y.myint".attr) + + // Check to make sure that the attributes from either side of the join have unique expression + // ids. + query.queryExecution.analyzed.output.filter(_.name == "myint") match { + case Seq(i1, i2) if(i1.exprId == i2.exprId) => + fail(s"Duplicate expression IDs found in query plan: $query") + case Seq(_, _) => // All good + } + + // TODO: We can't run this query as it NPEs + } + test("Import of simple Parquet file") { val result = getRDD(ParquetTestData.testData).collect() assert(result.size === 15) From df1b9f7b1a07bf8d806695a7684f9d69bf705093 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 29 Mar 2014 23:03:03 -0700 Subject: [PATCH 25/27] SPARK-1336 Reducing the output of run-tests script. Author: Prashant Sharma Author: Prashant Sharma Closes #262 from ScrapCodes/SPARK-1336/ReduceVerbosity and squashes the following commits: 87dfa54 [Prashant Sharma] Further reduction in noise and made pyspark tests to fail fast. 811170f [Prashant Sharma] Reducing the ouput of run-tests script. --- .gitignore | 1 + dev/run-tests | 7 +++---- dev/scalastyle | 27 +++++++++++++++++++++++++++ python/run-tests | 19 ++++++++++++------- 4 files changed, 43 insertions(+), 11 deletions(-) create mode 100755 dev/scalastyle diff --git a/.gitignore b/.gitignore index 3a68abd955b22..cd9f90d55932c 100644 --- a/.gitignore +++ b/.gitignore @@ -47,3 +47,4 @@ spark-*-bin.tar.gz unit-tests.log /lib/ rat-results.txt +scalastyle.txt diff --git a/dev/run-tests b/dev/run-tests index 6f115d2abd5b0..a6fcc40a5ba6e 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -39,18 +39,17 @@ JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..* echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" - dev/check-license echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" -sbt/sbt clean scalastyle +dev/scalastyle echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -sbt/sbt assembly test +sbt/sbt assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" echo "=========================================================================" echo "Running PySpark tests" @@ -64,5 +63,5 @@ echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore -sbt/sbt mima-report-binary-issues +sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" diff --git a/dev/scalastyle b/dev/scalastyle new file mode 100755 index 0000000000000..5a18f4d672825 --- /dev/null +++ b/dev/scalastyle @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +# +# 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. +# + +sbt/sbt clean scalastyle > scalastyle.txt +ERRORS=$(cat scalastyle.txt | grep -e "error file") +if test ! -z "$ERRORS"; then + echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" + exit 1 +else + echo -e "Scalastyle checks passed.\n" +fi diff --git a/python/run-tests b/python/run-tests index a986ac9380be4..b2b60f08b48e2 100755 --- a/python/run-tests +++ b/python/run-tests @@ -29,8 +29,18 @@ FAILED=0 rm -f unit-tests.log function run_test() { - SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log + SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a > unit-tests.log FAILED=$((PIPESTATUS[0]||$FAILED)) + + # Fail and exit on the first test failure. + if [[ $FAILED != 0 ]]; then + cat unit-tests.log | grep -v "^[0-9][0-9]*" # filter all lines starting with a number. + echo -en "\033[31m" # Red + echo "Had test failures; see logs." + echo -en "\033[0m" # No color + exit -1 + fi + } run_test "pyspark/rdd.py" @@ -46,12 +56,7 @@ run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" -if [[ $FAILED != 0 ]]; then - echo -en "\033[31m" # Red - echo "Had test failures; see logs." - echo -en "\033[0m" # No color - exit -1 -else +if [[ $FAILED == 0 ]]; then echo -en "\033[32m" # Green echo "Tests passed." echo -en "\033[0m" # No color From 95d7d2a3fc2adc0bbca90d015c6ca319fffb26aa Mon Sep 17 00:00:00 2001 From: jerryshao Date: Sun, 30 Mar 2014 10:03:58 -0700 Subject: [PATCH 26/27] [SPARK-1354][SQL] Add tableName as a qualifier for SimpleCatelogy Fix attribute unresolved when query with table name as a qualifier in SQLContext with SimplCatelog, details please see [SPARK-1354](https://issues.apache.org/jira/browse/SPARK-1354?jql=project%20%3D%20SPARK). Author: jerryshao Closes #272 from jerryshao/qualifier-fix and squashes the following commits: 7950170 [jerryshao] Add tableName as a qualifier for SimpleCatelogy --- .../spark/sql/catalyst/analysis/Catalog.scala | 3 ++- .../org/apache/spark/sql/SQLQuerySuite.scala | 27 +++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) 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 ff66177a03b8c..e09182dd8d5df 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 @@ -47,10 +47,11 @@ class SimpleCatalog extends Catalog { 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) // 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, table)).getOrElse(table) + alias.map(a => Subquery(a.toLowerCase, tableWithQualifiers)).getOrElse(tableWithQualifiers) } } 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 fa4a1d5189ea6..4c4fd6dbbedb4 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 @@ -216,4 +216,31 @@ class SQLQuerySuite extends QueryTest { (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) } + + test("select with table name as qualifier") { + checkAnswer( + sql("SELECT testData.value FROM testData WHERE testData.key = 1"), + Seq(Seq("1"))) + } + + test("inner join ON with table name as qualifier") { + checkAnswer( + sql("SELECT * FROM upperCaseData JOIN lowerCaseData ON lowerCaseData.n = upperCaseData.N"), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d"))) + } + + test("qualified select with inner join ON with table name as qualifier") { + checkAnswer( + sql("SELECT upperCaseData.N, upperCaseData.L FROM upperCaseData JOIN lowerCaseData " + + "ON lowerCaseData.n = upperCaseData.N"), + Seq( + (1, "A"), + (2, "B"), + (3, "C"), + (4, "D"))) + } } From d666053679ded5a32088c1758d20101126e23af6 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 30 Mar 2014 10:06:56 -0700 Subject: [PATCH 27/27] SPARK-1352 - Comment style single space before ending */ check. Author: Prashant Sharma Closes #261 from ScrapCodes/comment-style-check2 and squashes the following commits: 6cde61e [Prashant Sharma] comment style space before ending */ check. --- .../org/apache/spark/network/Connection.scala | 12 ++++++------ .../apache/spark/network/ConnectionManager.scala | 14 +++++++------- .../spark/network/ConnectionManagerTest.scala | 4 ++-- .../org/apache/spark/network/ReceiverTest.scala | 2 +- .../org/apache/spark/network/SenderTest.scala | 2 +- .../scala/org/apache/spark/ui/jobs/IndexPage.scala | 2 +- .../scala/org/apache/spark/util/MutablePair.scala | 4 ++-- .../examples/clickstream/PageViewGenerator.scala | 2 +- .../spark/streaming/flume/FlumeInputDStream.scala | 2 +- .../spark/graphx/impl/MessageToPartition.scala | 2 +- project/project/SparkPluginBuild.scala | 3 +-- ...la => SparkSpaceAfterCommentStartChecker.scala} | 8 +++++--- .../apache/spark/sql/parquet/ParquetRelation.scala | 6 +++--- 13 files changed, 32 insertions(+), 31 deletions(-) rename project/spark-style/src/main/scala/org/apache/spark/scalastyle/{SparkSpaceAfterCommentStyleCheck.scala => SparkSpaceAfterCommentStartChecker.scala} (89%) diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 16bd00fd189ff..2f7576c53b482 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -211,7 +211,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, def addMessage(message: Message) { messages.synchronized{ - /* messages += message*/ + /* messages += message */ messages.enqueue(message) logDebug("Added [" + message + "] to outbox for sending to " + "[" + getRemoteConnectionManagerId() + "]") @@ -222,7 +222,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, messages.synchronized { while (!messages.isEmpty) { /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ - /* val message = messages(nextMessageToBeUsed)*/ + /* val message = messages(nextMessageToBeUsed) */ val message = messages.dequeue val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { @@ -262,7 +262,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, val currentBuffers = new ArrayBuffer[ByteBuffer]() - /* channel.socket.setSendBufferSize(256 * 1024)*/ + /* channel.socket.setSendBufferSize(256 * 1024) */ override def getRemoteAddress() = address @@ -355,7 +355,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } case None => { // changeConnectionKeyInterest(0) - /* key.interestOps(0)*/ + /* key.interestOps(0) */ return false } } @@ -540,10 +540,10 @@ private[spark] class ReceivingConnection( return false } - /* logDebug("Read " + bytesRead + " bytes for the buffer")*/ + /* logDebug("Read " + bytesRead + " bytes for the buffer") */ if (currentChunk.buffer.remaining == 0) { - /* println("Filled buffer at " + System.currentTimeMillis)*/ + /* println("Filled buffer at " + System.currentTimeMillis) */ val bufferMessage = inbox.getMessageForChunk(currentChunk).get if (bufferMessage.isCompletelyReceived) { bufferMessage.flip diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 2682f9d0ed7f0..6b0a972f0bbe0 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -505,7 +505,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } handleMessageExecutor.execute(runnable) - /* handleMessage(connection, message)*/ + /* handleMessage(connection, message) */ } private def handleClientAuthentication( @@ -859,14 +859,14 @@ private[spark] object ConnectionManager { None }) - /* testSequentialSending(manager)*/ - /* System.gc()*/ + /* testSequentialSending(manager) */ + /* System.gc() */ - /* testParallelSending(manager)*/ - /* System.gc()*/ + /* testParallelSending(manager) */ + /* System.gc() */ - /* testParallelDecreasingSending(manager)*/ - /* System.gc()*/ + /* testParallelDecreasingSending(manager) */ + /* System.gc() */ testContinuousSending(manager) System.gc() diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index e5745d7daa153..9d9b9dbdd5331 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -47,8 +47,8 @@ private[spark] object ConnectionManagerTest extends Logging{ val slaves = slavesFile.mkString.split("\n") slavesFile.close() - /* println("Slaves")*/ - /* slaves.foreach(println)*/ + /* println("Slaves") */ + /* slaves.foreach(println) */ val tasknum = if (args.length > 2) args(2).toInt else slaves.length val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 17fd931c9f075..2b41c403b2e0a 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -27,7 +27,7 @@ private[spark] object ReceiverTest { println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ + /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ val buffer = ByteBuffer.wrap("response".getBytes) Some(Message.createBufferMessage(buffer, msg.id)) }) diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 905eddfbb9450..14c094c6177d5 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -50,7 +50,7 @@ private[spark] object SenderTest { (0 until count).foreach(i => { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis - /* println("Started timer at " + startTime)*/ + /* println("Started timer at " + startTime) */ val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index f3c93d4214ad0..70d62b66a4829 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils -/** Page showing list of all ongoing and recently finished stages and pools*/ +/** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class IndexPage(parent: JobProgressUI) { private val appName = parent.appName private val basePath = parent.basePath diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index a898824cff0ca..a6b39247a54ca 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -24,8 +24,8 @@ package org.apache.spark.util * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ -case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T1, - @specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T2] +case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, + @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) extends Product2[T1, T2] { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index 0ac46c31c24c8..251f65fe4df9c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -21,7 +21,7 @@ import java.net.ServerSocket import java.io.PrintWriter import util.Random -/** Represents a page view on a website with associated dimension data.*/ +/** Represents a page view on a website with associated dimension data. */ class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) extends Serializable { override def toString() : String = { diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index ce3ef47cfe4bc..34012b846e21e 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -127,7 +127,7 @@ class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { } /** A NetworkReceiver which listens for events using the - * Flume Avro interface.*/ + * Flume Avro interface. */ private[streaming] class FlumeReceiver( host: String, diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index bebe3740bc6c0..9d4f3750cb8e4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -45,7 +45,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( * @param data value to send */ private[graphx] -class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T]( +class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T]( @transient var partition: PartitionID, var data: T) extends Product2[PartitionID, T] with Serializable { diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index 43361aa2b4c41..5a307044ba123 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -34,8 +34,7 @@ object SparkPluginDef extends Build { version := sparkVersion, scalaVersion := "2.10.3", scalacOptions := Seq("-unchecked", "-deprecation"), - libraryDependencies ++= Dependencies.scalaStyle, - sbtPlugin := true + libraryDependencies ++= Dependencies.scalaStyle ) object Dependencies { diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala similarity index 89% rename from project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala rename to project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala index 2f3c1a182814d..80d3faa3fe749 100644 --- a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala +++ b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala @@ -25,13 +25,15 @@ import scalariform.lexer.{MultiLineComment, ScalaDocComment, SingleLineComment, import scalariform.parser.CompilationUnit class SparkSpaceAfterCommentStartChecker extends ScalariformChecker { - val errorKey: String = "insert.a.single.space.after.comment.start" + val errorKey: String = "insert.a.single.space.after.comment.start.and.before.end" private def multiLineCommentRegex(comment: Token) = - Pattern.compile( """/\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + Pattern.compile( """/\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() || + Pattern.compile( """/\*.*\S\*/""", Pattern.DOTALL).matcher(comment.text.trim).matches() private def scalaDocPatternRegex(comment: Token) = - Pattern.compile( """/\*\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + Pattern.compile( """/\*\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() || + Pattern.compile( """/\*\*.*\S\*/""", Pattern.DOTALL).matcher(comment.text.trim).matches() private def singleLineCommentRegex(comment: Token): Boolean = comment.text.trim.matches( """//\S+.*""") && !comment.text.trim.matches( """///+""") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 67a34e1f21cc7..4ab755c096bd8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -57,19 +57,19 @@ import org.apache.spark.sql.catalyst.types._ case class ParquetRelation(tableName: String, path: String) extends BaseRelation with MultiInstanceRelation { - /** Schema derived from ParquetFile **/ + /** Schema derived from ParquetFile */ def parquetSchema: MessageType = ParquetTypesConverter .readMetaData(new Path(path)) .getFileMetaData .getSchema - /** Attributes **/ + /** Attributes */ val attributes = ParquetTypesConverter .convertToAttributes(parquetSchema) - /** Output **/ + /** Output */ override val output = attributes // Parquet files have no concepts of keys, therefore no Partitioner