Skip to content

Commit

Permalink
[SPARK-30012][CORE][SQL] Change classes extending scala collection cl…
Browse files Browse the repository at this point in the history
…asses to work with 2.13

### What changes were proposed in this pull request?

Move some classes extending Scala collections into parallel source trees, to support 2.13; other minor collection-related modifications.

Modify some classes extending Scala collections to work with 2.13 as well as 2.12. In many cases, this means introducing parallel source trees, as the type hierarchy changed in ways that one class can't support both.

### Why are the changes needed?

To support building for Scala 2.13 in the future.

### Does this PR introduce any user-facing change?

There should be no behavior change.

### How was this patch tested?

Existing tests. Note that the 2.13 changes are not tested by the PR builder, of course. They compile in 2.13 but can't even be tested locally. Later, once the project can be compiled for 2.13, thus tested, it's possible the 2.13 implementations will need updates.

Closes #26728 from srowen/SPARK-30012.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
  • Loading branch information
srowen authored and dongjoon-hyun committed Dec 3, 2019
1 parent a3394e4 commit 4193d2f
Show file tree
Hide file tree
Showing 19 changed files with 510 additions and 10 deletions.
3 changes: 1 addition & 2 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@

<properties>
<sbt.project.name>core</sbt.project.name>
<extra.source.dir>src/main/scala-${scala.binary.version}</extra.source.dir>
</properties>

<dependencies>
Expand Down Expand Up @@ -530,7 +529,7 @@
</goals>
<configuration>
<sources>
<source>${extra.source.dir}</source>
<source>src/main/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ import scala.collection.generic.Growable
private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A])
extends Iterable[A] with Growable[A] with Serializable {

// Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation.

private val underlying = new JPriorityQueue[A](maxSize, ord)

override def iterator: Iterator[A] = underlying.iterator.asScala
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@ private[spark] case class TimeStampedValue[V](value: V, timestamp: Long)
private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false)
extends mutable.Map[A, B]() with Logging {

// Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation.

private val internalMap = new ConcurrentHashMap[A, TimeStampedValue[B]]()

def get(key: A): Option[B] = {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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

import java.io.Serializable
import java.util.{PriorityQueue => JPriorityQueue}

import scala.collection.JavaConverters._
import scala.collection.mutable.Growable

/**
* Bounded priority queue. This class wraps the original PriorityQueue
* class and modifies it such that only the top K elements are retained.
* The top K elements are defined by an implicit Ordering[A].
*/
private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A])
extends Iterable[A] with Growable[A] with Serializable {

// Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation.

private val underlying = new JPriorityQueue[A](maxSize, ord)

override def iterator: Iterator[A] = underlying.iterator.asScala

override def size: Int = underlying.size

override def knownSize: Int = size

override def addAll(xs: IterableOnce[A]): this.type = {
xs.foreach { this += _ }
this
}

override def addOne(elem: A): this.type = {
if (size < maxSize) {
underlying.offer(elem)
} else {
maybeReplaceLowest(elem)
}
this
}

def poll(): A = {
underlying.poll()
}

override def clear(): Unit = { underlying.clear() }

private def maybeReplaceLowest(a: A): Boolean = {
val head = underlying.peek()
if (head != null && ord.gt(a, head)) {
underlying.poll()
underlying.offer(a)
} else {
false
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
/*
* 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

import java.util.Map.Entry
import java.util.Set
import java.util.concurrent.ConcurrentHashMap

import scala.collection.JavaConverters._
import scala.collection.mutable

import org.apache.spark.internal.Logging

private[spark] case class TimeStampedValue[V](value: V, timestamp: Long)

/**
* This is a custom implementation of scala.collection.mutable.Map which stores the insertion
* timestamp along with each key-value pair. If specified, the timestamp of each pair can be
* updated every time it is accessed. Key-value pairs whose timestamp are older than a particular
* threshold time can then be removed using the clearOldValues method. This is intended to
* be a drop-in replacement of scala.collection.mutable.HashMap.
*
* @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed
*/
private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false)
extends mutable.Map[A, B]() with Logging {

// Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation.

private val internalMap = new ConcurrentHashMap[A, TimeStampedValue[B]]()

def get(key: A): Option[B] = {
val value = internalMap.get(key)
if (value != null && updateTimeStampOnGet) {
internalMap.replace(key, value, TimeStampedValue(value.value, currentTime))
}
Option(value).map(_.value)
}

def iterator: Iterator[(A, B)] = {
getEntrySet.iterator.asScala.map(kv => (kv.getKey, kv.getValue.value))
}

def getEntrySet: Set[Entry[A, TimeStampedValue[B]]] = internalMap.entrySet

override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = {
val newMap = new TimeStampedHashMap[A, B1]
val oldInternalMap = this.internalMap.asInstanceOf[ConcurrentHashMap[A, TimeStampedValue[B1]]]
newMap.internalMap.putAll(oldInternalMap)
kv match { case (a, b) => newMap.internalMap.put(a, TimeStampedValue(b, currentTime)) }
newMap
}

override def addOne(kv: (A, B)): this.type = {
kv match { case (a, b) => internalMap.put(a, TimeStampedValue(b, currentTime)) }
this
}

override def subtractOne(key: A): this.type = {
internalMap.remove(key)
this
}

override def update(key: A, value: B): Unit = {
this += ((key, value))
}

override def apply(key: A): B = {
get(key).getOrElse { throw new NoSuchElementException() }
}

override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = {
internalMap.asScala.map { case (k, TimeStampedValue(v, t)) => (k, v) }.filter(p)
}

override def empty: mutable.Map[A, B] = new TimeStampedHashMap[A, B]()

override def size: Int = internalMap.size

override def foreach[U](f: ((A, B)) => U): Unit = {
val it = getEntrySet.iterator
while(it.hasNext) {
val entry = it.next()
val kv = (entry.getKey, entry.getValue.value)
f(kv)
}
}

def putIfAbsent(key: A, value: B): Option[B] = {
val prev = internalMap.putIfAbsent(key, TimeStampedValue(value, currentTime))
Option(prev).map(_.value)
}

def putAll(map: Map[A, B]): Unit = {
map.foreach { case (k, v) => update(k, v) }
}

def toMap: Map[A, B] = iterator.toMap

def clearOldValues(threshTime: Long, f: (A, B) => Unit): Unit = {
val it = getEntrySet.iterator
while (it.hasNext) {
val entry = it.next()
if (entry.getValue.timestamp < threshTime) {
f(entry.getKey, entry.getValue.value)
logDebug("Removing key " + entry.getKey)
it.remove()
}
}
}

/** Removes old key-value pairs that have timestamp earlier than `threshTime`. */
def clearOldValues(threshTime: Long): Unit = {
clearOldValues(threshTime, (_, _) => ())
}

private def currentTime: Long = System.currentTimeMillis

// For testing

def getTimeStampedValue(key: A): Option[TimeStampedValue[B]] = {
Option(internalMap.get(key))
}

def getTimestamp(key: A): Option[Long] = {
getTimeStampedValue(key).map(_.timestamp)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -112,8 +112,6 @@ private[spark] class CompactBuffer[T: ClassTag] extends Seq[T] with Serializable

override def length: Int = curSize

override def size: Int = curSize

override def iterator: Iterator[T] = new Iterator[T] {
private var pos = 0
override def hasNext: Boolean = pos < curSize
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -367,7 +367,7 @@ class ExternalAppendOnlyMap[K, V, C](
private def removeFromBuffer[T](buffer: ArrayBuffer[T], index: Int): T = {
val elem = buffer(index)
buffer(index) = buffer(buffer.size - 1) // This also works if index == buffer.size - 1
buffer.reduceToSize(buffer.size - 1)
buffer.trimEnd(1)
elem
}

Expand Down
2 changes: 1 addition & 1 deletion dev/change-scala-version.sh
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

set -e

VALID_VERSIONS=( 2.12 )
VALID_VERSIONS=( 2.12 2.13 )

usage() {
echo "Usage: $(basename $0) [-h|--help] <version>
Expand Down
6 changes: 2 additions & 4 deletions repl/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,6 @@

<properties>
<sbt.project.name>repl</sbt.project.name>
<extra.source.dir>src/main/scala-${scala.binary.version}</extra.source.dir>
<extra.testsource.dir>src/test/scala-${scala.binary.version}</extra.testsource.dir>
</properties>

<dependencies>
Expand Down Expand Up @@ -146,7 +144,7 @@
</goals>
<configuration>
<sources>
<source>${extra.source.dir}</source>
<source>src/main/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>
Expand All @@ -158,7 +156,7 @@
</goals>
<configuration>
<sources>
<source>${extra.testsource.dir}</source>
<source>src/test/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>
Expand Down
18 changes: 18 additions & 0 deletions sql/catalyst/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,24 @@
<treatWarningsAsErrors>true</treatWarningsAsErrors>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
<id>add-sources</id>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ object AttributeMap {
class AttributeMap[A](val baseMap: Map[ExprId, (Attribute, A)])
extends Map[Attribute, A] with Serializable {

// Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation.

override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2)

override def contains(k: Attribute): Boolean = get(k).isDefined
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ class ExpressionSet protected(
protected val originals: mutable.Buffer[Expression] = new ArrayBuffer)
extends Set[Expression] {

// Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation.

protected def add(e: Expression): Unit = {
if (!e.deterministic) {
originals += e
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@ import java.util.Locale
class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T]
with Serializable {

// Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation.

val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT)))

override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase(Locale.ROOT))
Expand Down
Loading

0 comments on commit 4193d2f

Please sign in to comment.