diff options
author | hyukjinkwon <gurwls223@gmail.com> | 2016-04-16 14:56:23 +0100 |
---|---|---|
committer | Sean Owen <sowen@cloudera.com> | 2016-04-16 14:56:23 +0100 |
commit | 9f678e97549b19d6d979b22fa4079094ce9fb2c0 (patch) | |
tree | 978e18e46d294ba336f4657c6d627d146680f915 /sql/core/src/main/scala/org/apache | |
parent | 527c780bb0d6cb074128448da00cb330e9049385 (diff) | |
download | spark-9f678e97549b19d6d979b22fa4079094ce9fb2c0.tar.gz spark-9f678e97549b19d6d979b22fa4079094ce9fb2c0.tar.bz2 spark-9f678e97549b19d6d979b22fa4079094ce9fb2c0.zip |
[MINOR] Remove inappropriate type notation and extra anonymous closure within functional transformations
## What changes were proposed in this pull request?
This PR removes
- Inappropriate type notations
For example, from
```scala
words.foreachRDD { (rdd: RDD[String], time: Time) =>
...
```
to
```scala
words.foreachRDD { (rdd, time) =>
...
```
- Extra anonymous closure within functional transformations.
For example,
```scala
.map(item => {
...
})
```
which can be just simply as below:
```scala
.map { item =>
...
}
```
and corrects some obvious style nits.
## How was this patch tested?
This was tested after adding rules in `scalastyle-config.xml`, which ended up with not finding all perfectly.
The rules applied were below:
- For the first correction,
```xml
<check customId="NoExtraClosure" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
<parameters><parameter name="regex">(?m)\.[a-zA-Z_][a-zA-Z0-9]*\(\s*[^,]+s*=>\s*\{[^\}]+\}\s*\)</parameter></parameters>
</check>
```
```xml
<check customId="NoExtraClosure" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
<parameters><parameter name="regex">\.[a-zA-Z_][a-zA-Z0-9]*\s*[\{|\(]([^\n>,]+=>)?\s*\{([^()]|(?R))*\}^[,]</parameter></parameters>
</check>
```
- For the second correction
```xml
<check customId="TypeNotation" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
<parameters><parameter name="regex">\.[a-zA-Z_][a-zA-Z0-9]*\s*[\{|\(]\s*\([^):]*:R))*\}^[,]</parameter></parameters>
</check>
```
**Those rules were not added**
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #12413 from HyukjinKwon/SPARK-style.
Diffstat (limited to 'sql/core/src/main/scala/org/apache')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala | 4 | ||||
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala | 2 |
2 files changed, 2 insertions, 4 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala index 0c3e3c3fc1..f021f3758c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -60,9 +60,7 @@ case class ShuffledHashJoin( val context = TaskContext.get() val relation = HashedRelation(iter, buildKeys, taskMemoryManager = context.taskMemoryManager()) // This relation is usually used until the end of task. - context.addTaskCompletionListener((t: TaskContext) => - relation.close() - ) + context.addTaskCompletionListener(_ => relation.close()) relation } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index d603f63a08..9afbd0e994 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -431,7 +431,7 @@ private[sql] object StatFunctions extends Logging { s"exceed 1e4. Currently $columnSize") val table = counts.groupBy(_.get(0)).map { case (col1Item, rows) => val countsRow = new GenericMutableRow(columnSize + 1) - rows.foreach { (row: Row) => + rows.foreach { row => // row.get(0) is column 1 // row.get(1) is column 2 // row.get(2) is the frequency |