[SPARK-30012][CORE][SQL] Change classes extending scala collection classes 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>
This commit is contained in:
Sean Owen 2019-12-03 08:59:43 -08:00 committed by Dongjoon Hyun
parent a3394e49a7
commit 4193d2f4cc
19 changed files with 510 additions and 10 deletions

View file

@ -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>
@ -530,7 +529,7 @@
</goals>
<configuration>
<sources>
<source>${extra.source.dir}</source>
<source>src/main/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>

View file

@ -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

View file

@ -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] = {

View file

@ -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
}
}
}

View file

@ -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)
}
}

View file

@ -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

View file

@ -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
}

View file

@ -19,7 +19,7 @@
set -e
VALID_VERSIONS=( 2.12 )
VALID_VERSIONS=( 2.12 2.13 )
usage() {
echo "Usage: $(basename $0) [-h|--help] <version>

View file

@ -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>
@ -146,7 +144,7 @@
</goals>
<configuration>
<sources>
<source>${extra.source.dir}</source>
<source>src/main/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>
@ -158,7 +156,7 @@
</goals>
<configuration>
<sources>
<source>${extra.testsource.dir}</source>
<source>src/test/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>

View file

@ -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>

View file

@ -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

View file

@ -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

View file

@ -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))

View file

@ -0,0 +1,46 @@
/*
* 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.sql.catalyst.expressions
/**
* Builds a map that is keyed by an Attribute's expression id. Using the expression id allows values
* to be looked up even when the attributes used differ cosmetically (i.e., the capitalization
* of the name, or the expected nullability).
*/
object AttributeMap {
def apply[A](kvs: Seq[(Attribute, A)]): AttributeMap[A] = {
new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap)
}
}
class AttributeMap[A](val baseMap: Map[ExprId, (Attribute, A)])
extends Map[Attribute, A] with Serializable {
// Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation.
override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2)
override def contains(k: Attribute): Boolean = get(k).isDefined
override def updated[B1 >: A](key: Attribute, value: B1): Map[Attribute, B1] =
baseMap.values.toMap + (key -> value)
override def iterator: Iterator[(Attribute, A)] = baseMap.valuesIterator
override def removed(key: Attribute): Map[Attribute, A] = baseMap.values.toMap - key
}

View file

@ -0,0 +1,100 @@
/*
* 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.sql.catalyst.expressions
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
object ExpressionSet {
/** Constructs a new [[ExpressionSet]] by applying [[Canonicalize]] to `expressions`. */
def apply(expressions: TraversableOnce[Expression]): ExpressionSet = {
val set = new ExpressionSet()
expressions.foreach(set.add)
set
}
}
/**
* A [[Set]] where membership is determined based on determinacy and a canonical representation of
* an [[Expression]] (i.e. one that attempts to ignore cosmetic differences).
* See [[Canonicalize]] for more details.
*
* 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
* guaranteed to see at least one such expression. For example:
*
* {{{
* val set = ExpressionSet(a + 1, 1 + a)
*
* set.iterator => Iterator(a + 1)
* set.contains(a + 1) => true
* set.contains(1 + a) => true
* set.contains(a + 2) => false
* }}}
*
* For non-deterministic expressions, they are always considered as not contained in the [[Set]].
* On adding a non-deterministic expression, simply append it to the original expressions.
* This is consistent with how we define `semanticEquals` between two expressions.
*/
class ExpressionSet protected(
protected val baseSet: mutable.Set[Expression] = new mutable.HashSet,
protected val originals: mutable.Buffer[Expression] = new ArrayBuffer)
extends Set[Expression] {
// Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation.
protected def add(e: Expression): Unit = {
if (!e.deterministic) {
originals += e
} else if (!baseSet.contains(e.canonicalized) ) {
baseSet.add(e.canonicalized)
originals += e
}
}
override def contains(elem: Expression): Boolean = baseSet.contains(elem.canonicalized)
override def incl(elem: Expression): ExpressionSet = {
val newSet = new ExpressionSet(baseSet.clone(), originals.clone())
newSet.add(elem)
newSet
}
override def excl(elem: Expression): ExpressionSet = {
if (elem.deterministic) {
val newBaseSet = baseSet.clone().filterNot(_ == elem.canonicalized)
val newOriginals = originals.clone().filterNot(_.canonicalized == elem.canonicalized)
new ExpressionSet(newBaseSet, newOriginals)
} else {
new ExpressionSet(baseSet.clone(), originals.clone())
}
}
override def iterator: Iterator[Expression] = originals.iterator
/**
* Returns a string containing both the post [[Canonicalize]] expressions and the original
* expressions in this set.
*/
def toDebugString: String =
s"""
|baseSet: ${baseSet.mkString(", ")}
|originals: ${originals.mkString(", ")}
""".stripMargin
}

View file

@ -0,0 +1,59 @@
/*
* 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.sql.catalyst.util
import java.util.Locale
/**
* Builds a map in which keys are case insensitive. Input map can be accessed for cases where
* case-sensitive information is required. The primary constructor is marked private to avoid
* nested case-insensitive map creation, otherwise the keys in the original map will become
* case-insensitive in this scenario.
* Note: CaseInsensitiveMap is serializable. However, after transformation, e.g. `filterKeys()`,
* it may become not serializable.
*/
class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T]
with Serializable {
// Note: this class supports Scala 2.13. A parallel source tree has a 2.12 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))
override def contains(k: String): Boolean =
keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT))
override def updated[B1 >: T](key: String, value: B1): Map[String, B1] = {
new CaseInsensitiveMap[B1](originalMap + (key -> value))
}
override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator
override def removed(key: String): Map[String, T] = {
new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(key)))
}
}
object CaseInsensitiveMap {
def apply[T](params: Map[String, T]): CaseInsensitiveMap[T] = params match {
case caseSensitiveMap: CaseInsensitiveMap[T] => caseSensitiveMap
case _ => new CaseInsensitiveMap(params)
}
}

View file

@ -198,6 +198,7 @@
<sources>
<source>v${hive.version.short}/src/main/scala</source>
<source>v${hive.version.short}/src/main/java</source>
<source>src/main/scala-${scala.binary.version}</source>
</sources>
</configuration>
</execution>

View file

@ -29,6 +29,8 @@ class StreamProgress(
new immutable.HashMap[SparkDataStream, OffsetV2])
extends scala.collection.immutable.Map[SparkDataStream, OffsetV2] {
// Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation.
def toOffsetSeq(source: Seq[SparkDataStream], metadata: OffsetSeqMetadata): OffsetSeq = {
OffsetSeq(source.map(get), Some(metadata))
}

View file

@ -0,0 +1,53 @@
/*
* 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.sql.execution.streaming
import scala.collection.immutable
import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream}
/**
* A helper class that looks like a Map[Source, Offset].
*/
class StreamProgress(
val baseMap: immutable.Map[SparkDataStream, OffsetV2] =
new immutable.HashMap[SparkDataStream, OffsetV2])
extends scala.collection.immutable.Map[SparkDataStream, OffsetV2] {
// Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation.
def toOffsetSeq(source: Seq[SparkDataStream], metadata: OffsetSeqMetadata): OffsetSeq = {
OffsetSeq(source.map(get), Some(metadata))
}
override def toString: String =
baseMap.map { case (k, v) => s"$k: $v"}.mkString("{", ",", "}")
override def updated[B1 >: OffsetV2](key: SparkDataStream, value: B1): Map[SparkDataStream, B1] =
baseMap + (key -> value)
override def get(key: SparkDataStream): Option[OffsetV2] = baseMap.get(key)
override def iterator: Iterator[(SparkDataStream, OffsetV2)] = baseMap.iterator
override def removed(key: SparkDataStream): Map[SparkDataStream, OffsetV2] = baseMap - key
def ++(updates: IterableOnce[(SparkDataStream, OffsetV2)]): StreamProgress = {
new StreamProgress(baseMap ++ updates)
}
}