diff options
author | Dongjoon Hyun <dongjoon@apache.org> | 2016-03-14 09:07:39 +0000 |
---|---|---|
committer | Sean Owen <sowen@cloudera.com> | 2016-03-14 09:07:39 +0000 |
commit | acdf21970334cea9d6cfc287e4ccb8e72de9dee1 (patch) | |
tree | df8bcf3d80dc92ad74c5c27bd3618397205bcc86 /sql | |
parent | e58fa19d17db5dd8a00551e20b46921f98b958f7 (diff) | |
download | spark-acdf21970334cea9d6cfc287e4ccb8e72de9dee1.tar.gz spark-acdf21970334cea9d6cfc287e4ccb8e72de9dee1.tar.bz2 spark-acdf21970334cea9d6cfc287e4ccb8e72de9dee1.zip |
[MINOR][DOCS] Fix more typos in comments/strings.
## What changes were proposed in this pull request?
This PR fixes 135 typos over 107 files:
* 121 typos in comments
* 11 typos in testcase name
* 3 typos in log messages
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #11689 from dongjoon-hyun/fix_more_typos.
Diffstat (limited to 'sql')
39 files changed, 44 insertions, 44 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index f108264861..1219d4d453 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -305,7 +305,7 @@ trait Row extends Serializable { * @throws ClassCastException when data type does not match. */ def getStruct(i: Int): Row = { - // Product and Row both are recoginized as StructType in a Row + // Product and Row both are recognized as StructType in a Row val t = get(i) if (t.isInstanceOf[Product]) { Row.fromTuple(t.asInstanceOf[Product]) 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 c12b5c20ea..bf07f4557a 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 @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils */ object ScalaReflection extends ScalaReflection { val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - // Since we are creating a runtime mirror usign the class loader of current thread, + // Since we are creating a runtime mirror using the class loader of current thread, // we need to use def at here. So, every time we call mirror, it is using the // class loader of the current thread. // SPARK-13640: Synchronize this because universe.runtimeMirror is not thread-safe in Scala 2.10. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala index ad56c98649..9c38dd2ee4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala @@ -155,7 +155,7 @@ object DecimalPrecision extends Rule[LogicalPlan] { * * Note that technically this is an "optimization" and should go into the optimizer. However, * by the time the optimizer runs, these comparison expressions would be pretty hard to pattern - * match because there are multuple (at least 2) levels of casts involved. + * match because there are multiple (at least 2) levels of casts involved. * * There are a lot more possible rules we can implement, but we don't do them * because we are not sure how common they are. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala index 38c1641f73..2e30d83a60 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala @@ -96,7 +96,7 @@ import org.apache.spark.sql.types.IntegerType * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and * exchange operators. Keeping the number of distinct groups as low a possible should be priority, - * we could improve this in the current rule by applying more advanced expression cannocalization + * we could improve this in the current rule by applying more advanced expression canonicalization * techniques. */ object DistinctAggregationRewriter extends Rule[LogicalPlan] { 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 57bdb164e1..0f85f44ffa 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 @@ -307,7 +307,7 @@ object HiveTypeCoercion { case p @ Equality(left @ TimestampType(), right @ StringType()) => p.makeCopy(Array(left, Cast(right, TimestampType))) - // We should cast all relative timestamp/date/string comparison into string comparisions + // We should cast all relative timestamp/date/string comparison into string comparisons // This behaves as a user would expect because timestamp strings sort lexicographically. // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true case p @ BinaryComparison(left @ StringType(), right @ DateType()) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index 3831535574..8bdf9b29c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -53,7 +53,7 @@ object AttributeSet { * cosmetically (e.g., the names have different capitalizations). * * Note that we do not override equality for Attribute references as it is really weird when - * `AttributeReference("a"...) == AttrributeReference("b", ...)`. This tactic leads to broken tests, + * `AttributeReference("a"...) == AttributeReference("b", ...)`. This tactic leads to broken tests, * and also makes doing transformations hard (we always try keep older trees instead of new ones * when the transformation was a no-op). */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala index acea049adc..644a5b28a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -36,7 +36,7 @@ object ExpressionSet { * Internally this set uses the canonical representation, but keeps also track of the original * expressions to ease debugging. Since different expressions can share the same canonical * representation, this means that operations that extract expressions from this set are only - * guranteed to see at least one such expression. For example: + * guaranteed to see at least one such expression. For example: * * {{{ * val set = AttributeSet(a + 1, 1 + a) 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 22184f1ddf..500ff447a9 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 @@ -973,7 +973,7 @@ case class ScalaUDF( // scalastyle:on line.size.limit - // Generate codes used to convert the arguments to Scala type for user-defined funtions + // Generate codes used to convert the arguments to Scala type for user-defined functions private[this] def genCodeForConverter(ctx: CodegenContext, index: Int): String = { val converterClassName = classOf[Any => Any].getName val typeConvertersClassName = CatalystTypeConverters.getClass.getName + ".MODULE$" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index c4265a7539..3dbe634898 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -126,7 +126,7 @@ class CodegenContext { * For expressions that appear more than once, generate additional code to prevent * recomputing the value. * - * For example, consider two exprsesion generated from this SQL statement: + * For example, consider two expression generated from this SQL statement: * SELECT (col1 + col2), (col1 + col2) / col3. * * equivalentExpressions will match the tree containing `col1 + col2` and it will only @@ -140,7 +140,7 @@ class CodegenContext { // Foreach expression that is participating in subexpression elimination, the state to use. val subExprEliminationExprs = mutable.HashMap.empty[Expression, SubExprEliminationState] - // The collection of sub-exression result resetting methods that need to be called on each row. + // The collection of sub-expression result resetting methods that need to be called on each row. val subexprFunctions = mutable.ArrayBuffer.empty[String] def declareAddedFunctions(): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 5ceb36513f..103ab365e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -214,7 +214,7 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E /** Factory methods for CaseWhen. */ object CaseWhen { - // The maxium number of switches supported with codegen. + // The maximum number of switches supported with codegen. val MAX_NUM_CASES_FOR_CODEGEN = 20 def apply(branches: Seq[(Expression, Expression)], elseValue: Expression): CaseWhen = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala index b95c5dd892..7eba617fcd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala @@ -364,7 +364,7 @@ object MapObjects { * used as input for the `lambdaFunction`. It also carries the element type info. * @param lambdaFunction A function that take the `loopVar` as input, and used as lambda function * to handle collection elements. - * @param inputData An expression that when evaluted returns a collection object. + * @param inputData An expression that when evaluated returns a collection object. */ case class MapObjects private( loopVar: LambdaVariable, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index e4417e0955..da90ddbd63 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -66,7 +66,7 @@ object NumberConverter { * negative digit is found, ignore the suffix starting there. * * @param radix must be between MIN_RADIX and MAX_RADIX - * @param fromPos is the first element that should be conisdered + * @param fromPos is the first element that should be considered * @return the result should be treated as an unsigned 64-bit integer. */ private def encode(radix: Int, fromPos: Int): Long = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index de9a56dc9c..4e7bbc38d6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -276,7 +276,7 @@ class AnalysisErrorSuite extends AnalysisTest { test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) - // Since we manually construct the logical plan at here and Sum only accetp + // Since we manually construct the logical plan at here and Sum only accept // LongType, DoubleType, and DecimalType. We use LongType as the type of a. val plan = Aggregate( 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 aa1d2b0861..8b568b6dd6 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 @@ -250,7 +250,7 @@ class AnalysisSuite extends AnalysisTest { assertAnalysisSuccess(plan) } - test("SPARK-8654: different types in inlist but can be converted to a commmon type") { + test("SPARK-8654: different types in inlist but can be converted to a common type") { val plan = Project(Alias(In(Literal(null), Seq(Literal(1), Literal(1.2345))), "a")() :: Nil, LocalRelation() ) 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 c30434a006..6f289dcc47 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 @@ -205,7 +205,7 @@ class HiveTypeCoercionSuite extends PlanTest { Project(Seq(Alias(transformed, "a")()), testRelation)) } - test("cast NullType for expresions that implement ExpectsInputTypes") { + test("cast NullType for expressions that implement ExpectsInputTypes") { import HiveTypeCoercionSuite._ ruleTest(HiveTypeCoercion.ImplicitTypeCasts, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala index ce42e5784c..0b350c6a98 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala @@ -70,7 +70,7 @@ class ExpressionSetSuite extends SparkFunSuite { // Not commutative setTest(2, aUpper - bUpper, bUpper - aUpper) - // Reversable + // Reversible setTest(1, aUpper > bUpper, bUpper < aUpper) setTest(1, aUpper >= bUpper, bUpper <= aUpper) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala index 0dbfb01e88..f5374229ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala @@ -131,7 +131,7 @@ class HyperLogLogPlusPlusSuite extends SparkFunSuite { i += 1 } - // Merge the lower and upper halfs. + // Merge the lower and upper halves. hll.merge(buffer1a, buffer1b) // Create the other buffer in reverse diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index f7ba61d2b8..1751720a7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -78,7 +78,7 @@ class TypedColumn[-T, U]( * * {{{ * df("columnName") // On a specific DataFrame. - * col("columnName") // A generic column no yet associcated with a DataFrame. + * col("columnName") // A generic column no yet associated with a DataFrame. * col("columnName.field") // Extracting a struct field * col("`a.column.with.dots`") // Escape `.` in column names. * $"columnName" // Scala short hand for a named column. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala index 472ae716f1..a8700de135 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala @@ -223,7 +223,7 @@ class GroupedDataset[K, V] private[sql]( * Internal helper function for building typed aggregations that return tuples. For simplicity * and code reuse, we do this without the help of the type system and then use helper functions * that cast appropriately for the user facing interface. - * TODO: does not handle aggrecations that return nonflat results, + * TODO: does not handle aggregations that return nonflat results, */ protected def aggUntyped(columns: TypedColumn[_, _]*): Dataset[_] = { val encoders = columns.map(_.encoder) 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 3be4cce045..a92c99e06f 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 @@ -52,7 +52,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ protected def sparkContext = sqlContext.sparkContext // sqlContext will be null when we are being deserialized on the slaves. In this instance - // the value of subexpressionEliminationEnabled will be set by the desserializer after the + // the value of subexpressionEliminationEnabled will be set by the deserializer after the // constructor has run. val subexpressionEliminationEnabled: Boolean = if (sqlContext != null) { sqlContext.conf.subexpressionEliminationEnabled @@ -65,7 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ */ private val prepareCalled = new AtomicBoolean(false) - /** Overridden make copy also propogates sqlContext to copied plan. */ + /** Overridden make copy also propagates sqlContext to copied plan. */ override def makeCopy(newArgs: Array[AnyRef]): SparkPlan = { SQLContext.setActive(sqlContext) super.makeCopy(newArgs) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 3ec01185c4..f9d606e37e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -40,7 +40,7 @@ import org.apache.spark.unsafe.types.UTF8String * so we do not have helper methods for them. * * - * WARNNING: This only works with HeapByteBuffer + * WARNING: This only works with HeapByteBuffer */ private[columnar] object ByteBufferHelper { def getInt(buffer: ByteBuffer): Int = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index c3f8d7f75a..18a460fc85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -88,7 +88,7 @@ private[sql] object PartitioningUtils { }.unzip // We create pairs of (path -> path's partition value) here - // If the corresponding partition value is None, the pair will be skiped + // If the corresponding partition value is None, the pair will be skipped val pathsWithPartitionValues = paths.zip(partitionValues).flatMap(x => x._2.map(x._1 -> _)) if (pathsWithPartitionValues.isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index e295722cac..64a820c6d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -70,7 +70,7 @@ object JdbcUtils extends Logging { // Somewhat hacky, but there isn't a good way to identify whether a table exists for all // SQL database systems using JDBC meta data calls, considering "table" could also include - // the database name. Query used to find table exists can be overriden by the dialects. + // the database name. Query used to find table exists can be overridden by the dialects. Try { val statement = conn.prepareStatement(dialect.getTableExistsQuery(table)) try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala index c65a7bcff8..79e4491026 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.{StructField, StructType} /** - * A physical plan that evalutes a [[PythonUDF]], one partition of tuples at a time. + * A physical plan that evaluates a [[PythonUDF]], one partition of tuples at a time. * * Python evaluation works by sending the necessary (projected) input data via a socket to an * external Python process, and combine the result from the Python process with the original row. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala index 1bd71b6b02..e3b2d2f67e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala @@ -38,7 +38,7 @@ trait Sink { * Accepts a new batch of data as well as a [[Offset]] that denotes how far in the input * data computation has progressed to. When computation restarts after a failure, it is important * that a [[Sink]] returns the same [[Offset]] as the most recent batch of data that - * has been persisted durrably. Note that this does not necessarily have to be the + * has been persisted durably. Note that this does not necessarily have to be the * [[Offset]] for the most recent batch of data that was given to the sink. For example, * it is valid to buffer data before persisting, as long as the [[Offset]] is stored * transactionally as data is eventually persisted. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 096477ce0e..d7ff44afad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -100,7 +100,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) /** * A sink that stores the results in memory. This [[Sink]] is primarily intended for use in unit - * tests and does not provide durablility. + * tests and does not provide durability. */ class MemorySink(schema: StructType) extends Sink with Logging { /** An order list of batches that have been written to this [[Sink]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 384102e5ea..59429d254e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -441,7 +441,7 @@ object SQLConf { // NOTE: // // 1. Instead of SQLConf, this option *must be set in Hadoop Configuration*. - // 2. This option can be overriden by "spark.sql.parquet.output.committer.class". + // 2. This option can be overridden by "spark.sql.parquet.output.committer.class". val OUTPUT_COMMITTER_CLASS = stringConf("spark.sql.sources.outputCommitterClass", isPublic = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index e865dbe6b5..a7a826bc7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -80,7 +80,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { // Verify that the splits span the entire dataset assert(splits.flatMap(_.collect()).toSet == data.collect().toSet) - // Verify that the splits don't overalap + // Verify that the splits don't overlap assert(splits(0).intersect(splits(1)).collect().isEmpty) // Verify that the results are deterministic across multiple runs diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 9f32c8bf95..d7fa23651b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -46,7 +46,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { 1, 1, 1) } - test("SPARK-12404: Datatype Helper Serializablity") { + test("SPARK-12404: Datatype Helper Serializability") { val ds = sparkContext.parallelize(( new Timestamp(0), new Date(0), 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 182f287dd0..98d0008489 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 @@ -986,7 +986,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SET commands with illegal or inappropriate argument") { sqlContext.conf.clear() - // Set negative mapred.reduce.tasks for automatically determing + // Set negative mapred.reduce.tasks for automatically determining // the number of reducers is not supported intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala index 7a5b639115..81078dc6a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala @@ -80,7 +80,7 @@ trait StreamTest extends QueryTest with Timeouts { trait StreamMustBeRunning /** - * Adds the given data to the stream. Subsuquent check answers will block until this data has + * Adds the given data to the stream. Subsequent check answers will block until this data has * been processed. */ object AddData { @@ -109,7 +109,7 @@ trait StreamTest extends QueryTest with Timeouts { /** * Checks to make sure that the current data stored in the sink matches the `expectedAnswer`. - * This operation automatically blocks untill all added data has been processed. + * This operation automatically blocks until all added data has been processed. */ object CheckAnswer { def apply[A : Encoder](data: A*): CheckAnswerRows = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala index 7af3f94aef..3a7cb25b4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala @@ -80,7 +80,7 @@ class InferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(BooleanType, "\\N", "\\N") == BooleanType) } - test("Merging Nulltypes should yeild Nulltype.") { + test("Merging Nulltypes should yield Nulltype.") { val mergedNullTypes = CSVInferSchema.mergeRowTypes(Array(NullType), Array(NullType)) assert(mergedNullTypes.deep == Array(NullType).deep) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 1ef517324d..f66deea065 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -359,7 +359,7 @@ class JDBCSuite extends SparkFunSuite .collect().length === 3) } - test("Partioning on column that might have null values.") { + test("Partitioning on column that might have null values.") { assert( sqlContext.read.jdbc(urlWithUserAndPass, "TEST.EMP", "theid", 0, 4, 3, new Properties) .collect().length === 4) @@ -372,7 +372,7 @@ class JDBCSuite extends SparkFunSuite .collect().length === 4) } - test("SELECT * on partitioned table with a nullable partioncolumn") { + test("SELECT * on partitioned table with a nullable partition column") { assert(sql("SELECT * FROM nullparts").collect().size == 4) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 26c1ff5204..99f1661ad0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -339,7 +339,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { test("exceptions") { // Make sure we do throw correct exception when users use a relation provider that - // only implements the RelationProvier or the SchemaRelationProvider. + // only implements the RelationProvider or the SchemaRelationProvider. val schemaNotAllowed = intercept[Exception] { sql( """ diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 54fffb971d..694bd97515 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -68,7 +68,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { * with one of these strings is found, fail the test immediately. * The default value is `Seq("Error:")` * - * @param queriesAndExpectedAnswers one or more tupes of query + answer + * @param queriesAndExpectedAnswers one or more tuples of query + answer */ def runCliWithin( timeout: FiniteDuration, 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 56acb87c80..739fbaf444 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 @@ -245,7 +245,7 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") DropTable(tableName, ifExists.nonEmpty) - // Support "ANALYZE TABLE tableNmae COMPUTE STATISTICS noscan" + // Support "ANALYZE TABLE tableName COMPUTE STATISTICS noscan" case Token("TOK_ANALYZE", Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: isNoscan) => // Reference: @@ -535,7 +535,7 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging case Token("TOK_STORAGEHANDLER", _) => throw new AnalysisException( "CREATE TABLE AS SELECT cannot be used for a non-native table") - case _ => // Unsupport features + case _ => // Unsupported features } CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting.isDefined) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index 059ad8b1f7..8240f2f222 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -89,7 +89,7 @@ private[orc] object OrcFileOperator extends Logging { } def listOrcFiles(pathStr: String, conf: Configuration): Seq[Path] = { - // TODO: Check if the paths comming in are already qualified and simplify. + // TODO: Check if the paths coming in are already qualified and simplify. val origPath = new Path(pathStr) val fs = origPath.getFileSystem(conf) val path = origPath.makeQualified(fs.getUri, fs.getWorkingDirectory) 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 1053246fc2..5e452d107d 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 @@ -143,7 +143,7 @@ abstract class HiveComparisonTest 0D } - s"""SQLBuiler statistics: + s"""SQLBuilder statistics: |- Total query number: $numTotalQueries |- Number of convertible queries: $numConvertibleQueries |- Percentage of convertible queries: $percentage% 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 1002487447..d905f0cd68 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 @@ -602,7 +602,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |select * where key = 4 """.stripMargin) - // test get_json_object again Hive, because the HiveCompatabilitySuite cannot handle result + // test get_json_object again Hive, because the HiveCompatibilitySuite cannot handle result // with newline in it. createQueryTest("get_json_object #1", "SELECT get_json_object(src_json.json, '$') FROM src_json") |