Skip to content

Commit 2a80a4c

Browse files
committed
[SPARK-27106][SQL] merge CaseInsensitiveStringMap and DataSourceOptions
## What changes were proposed in this pull request? It's a little awkward to have 2 different classes(`CaseInsensitiveStringMap` and `DataSourceOptions`) to present the options in data source and catalog API. This PR merges these 2 classes, while keeping the name `CaseInsensitiveStringMap`, which is more precise. ## How was this patch tested? existing tests Closes #24025 from cloud-fan/option. Authored-by: Wenchen Fan <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 812ad55 commit 2a80a4c

File tree

63 files changed

+363
-558
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

63 files changed

+363
-558
lines changed

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -37,8 +37,7 @@ import org.apache.spark.sql.sources.v2.reader.streaming._
3737
* @param offsetReader a reader used to get kafka offsets. Note that the actual data will be
3838
* read by per-task consumers generated later.
3939
* @param kafkaParams String params for per-task Kafka consumers.
40-
* @param sourceOptions The [[org.apache.spark.sql.sources.v2.DataSourceOptions]] params which
41-
* are not Kafka consumer params.
40+
* @param sourceOptions Params which are not Kafka consumer params.
4241
* @param metadataPath Path to a directory this reader can use for writing metadata.
4342
* @param initialOffsets The Kafka offsets to start reading data at.
4443
* @param failOnDataLoss Flag indicating whether reading should fail in data loss

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,9 +33,9 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow
3333
import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset}
3434
import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchStream
3535
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
36-
import org.apache.spark.sql.sources.v2.DataSourceOptions
3736
import org.apache.spark.sql.sources.v2.reader._
3837
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset}
38+
import org.apache.spark.sql.util.CaseInsensitiveStringMap
3939
import org.apache.spark.util.UninterruptibleThread
4040

4141
/**
@@ -57,7 +57,7 @@ import org.apache.spark.util.UninterruptibleThread
5757
private[kafka010] class KafkaMicroBatchStream(
5858
kafkaOffsetReader: KafkaOffsetReader,
5959
executorKafkaParams: ju.Map[String, Object],
60-
options: DataSourceOptions,
60+
options: CaseInsensitiveStringMap,
6161
metadataPath: String,
6262
startingOffsets: KafkaOffsetRangeLimit,
6363
failOnDataLoss: Boolean) extends RateControlMicroBatchStream with Logging {
@@ -66,8 +66,7 @@ private[kafka010] class KafkaMicroBatchStream(
6666
"kafkaConsumer.pollTimeoutMs",
6767
SparkEnv.get.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 1000L)
6868

69-
private val maxOffsetsPerTrigger =
70-
Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong)
69+
private val maxOffsetsPerTrigger = Option(options.get("maxOffsetsPerTrigger")).map(_.toLong)
7170

7271
private val rangeCalculator = KafkaOffsetRangeCalculator(options)
7372

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.sql.kafka010
1919

2020
import org.apache.kafka.common.TopicPartition
2121

22-
import org.apache.spark.sql.sources.v2.DataSourceOptions
22+
import org.apache.spark.sql.util.CaseInsensitiveStringMap
2323

2424

2525
/**
@@ -91,8 +91,8 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int
9191

9292
private[kafka010] object KafkaOffsetRangeCalculator {
9393

94-
def apply(options: DataSourceOptions): KafkaOffsetRangeCalculator = {
95-
val optionalValue = Option(options.get("minPartitions").orElse(null)).map(_.toInt)
94+
def apply(options: CaseInsensitiveStringMap): KafkaOffsetRangeCalculator = {
95+
val optionalValue = Option(options.get("minPartitions")).map(_.toInt)
9696
new KafkaOffsetRangeCalculator(optionalValue)
9797
}
9898
}

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ import org.apache.spark.sql.sources.v2.writer.WriteBuilder
3838
import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite
3939
import org.apache.spark.sql.streaming.OutputMode
4040
import org.apache.spark.sql.types.StructType
41+
import org.apache.spark.sql.util.CaseInsensitiveStringMap
4142

4243
/**
4344
* The provider class for all Kafka readers and writers. It is designed such that it throws
@@ -103,8 +104,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
103104
failOnDataLoss(caseInsensitiveParams))
104105
}
105106

106-
override def getTable(options: DataSourceOptions): KafkaTable = {
107-
new KafkaTable(strategy(options.asMap().asScala.toMap))
107+
override def getTable(options: CaseInsensitiveStringMap): KafkaTable = {
108+
new KafkaTable(strategy(options.asScala.toMap))
108109
}
109110

110111
/**
@@ -358,11 +359,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
358359

359360
override def schema(): StructType = KafkaOffsetReader.kafkaSchema
360361

361-
override def newScanBuilder(options: DataSourceOptions): ScanBuilder = new ScanBuilder {
362+
override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = new ScanBuilder {
362363
override def build(): Scan = new KafkaScan(options)
363364
}
364365

365-
override def newWriteBuilder(options: DataSourceOptions): WriteBuilder = {
366+
override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = {
366367
new WriteBuilder {
367368
private var inputSchema: StructType = _
368369

@@ -375,20 +376,20 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
375376
import scala.collection.JavaConverters._
376377

377378
assert(inputSchema != null)
378-
val topic = Option(options.get(TOPIC_OPTION_KEY).orElse(null)).map(_.trim)
379-
val producerParams = kafkaParamsForProducer(options.asMap.asScala.toMap)
379+
val topic = Option(options.get(TOPIC_OPTION_KEY)).map(_.trim)
380+
val producerParams = kafkaParamsForProducer(options.asScala.toMap)
380381
new KafkaStreamingWrite(topic, producerParams, inputSchema)
381382
}
382383
}
383384
}
384385
}
385386

386-
class KafkaScan(options: DataSourceOptions) extends Scan {
387+
class KafkaScan(options: CaseInsensitiveStringMap) extends Scan {
387388

388389
override def readSchema(): StructType = KafkaOffsetReader.kafkaSchema
389390

390391
override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = {
391-
val parameters = options.asMap().asScala.toMap
392+
val parameters = options.asScala.toMap
392393
validateStreamOptions(parameters)
393394
// Each running query should use its own group id. Otherwise, the query may be only assigned
394395
// partial data since Kafka will assign partitions to multiple consumers having the same group
@@ -417,7 +418,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
417418
}
418419

419420
override def toContinuousStream(checkpointLocation: String): ContinuousStream = {
420-
val parameters = options.asMap().asScala.toMap
421+
val parameters = options.asScala.toMap
421422
validateStreamOptions(parameters)
422423
// Each running query should use its own group id. Otherwise, the query may be only assigned
423424
// partial data since Kafka will assign partitions to multiple consumers having the same group

external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -41,10 +41,10 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
4141
import org.apache.spark.sql.functions.{count, window}
4242
import org.apache.spark.sql.internal.SQLConf
4343
import org.apache.spark.sql.kafka010.KafkaSourceProvider._
44-
import org.apache.spark.sql.sources.v2.DataSourceOptions
4544
import org.apache.spark.sql.streaming.{StreamTest, Trigger}
4645
import org.apache.spark.sql.streaming.util.StreamManualClock
4746
import org.apache.spark.sql.test.SharedSQLContext
47+
import org.apache.spark.sql.util.CaseInsensitiveStringMap
4848

4949
abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with KafkaTest {
5050

@@ -1118,7 +1118,7 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase {
11181118
"kafka.bootstrap.servers" -> testUtils.brokerAddress,
11191119
"subscribe" -> topic
11201120
) ++ Option(minPartitions).map { p => "minPartitions" -> p}
1121-
val dsOptions = new DataSourceOptions(options.asJava)
1121+
val dsOptions = new CaseInsensitiveStringMap(options.asJava)
11221122
val table = provider.getTable(dsOptions)
11231123
val stream = table.newScanBuilder(dsOptions).build().toMicroBatchStream(dir.getAbsolutePath)
11241124
val inputPartitions = stream.planInputPartitions(

external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -22,21 +22,21 @@ import scala.collection.JavaConverters._
2222
import org.apache.kafka.common.TopicPartition
2323

2424
import org.apache.spark.SparkFunSuite
25-
import org.apache.spark.sql.sources.v2.DataSourceOptions
25+
import org.apache.spark.sql.util.CaseInsensitiveStringMap
2626

2727
class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite {
2828

2929
def testWithMinPartitions(name: String, minPartition: Int)
3030
(f: KafkaOffsetRangeCalculator => Unit): Unit = {
31-
val options = new DataSourceOptions(Map("minPartitions" -> minPartition.toString).asJava)
31+
val options = new CaseInsensitiveStringMap(Map("minPartitions" -> minPartition.toString).asJava)
3232
test(s"with minPartition = $minPartition: $name") {
3333
f(KafkaOffsetRangeCalculator(options))
3434
}
3535
}
3636

3737

3838
test("with no minPartition: N TopicPartitions to N offset ranges") {
39-
val calc = KafkaOffsetRangeCalculator(DataSourceOptions.empty())
39+
val calc = KafkaOffsetRangeCalculator(CaseInsensitiveStringMap.empty())
4040
assert(
4141
calc.getRanges(
4242
fromOffsets = Map(tp1 -> 1),
@@ -64,7 +64,7 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite {
6464
}
6565

6666
test("with no minPartition: empty ranges ignored") {
67-
val calc = KafkaOffsetRangeCalculator(DataSourceOptions.empty())
67+
val calc = KafkaOffsetRangeCalculator(CaseInsensitiveStringMap.empty())
6868
assert(
6969
calc.getRanges(
7070
fromOffsets = Map(tp1 -> 1, tp2 -> 1),

sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java

Lines changed: 58 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -31,19 +31,20 @@
3131
* This is used to pass options to v2 implementations to ensure consistent case insensitivity.
3232
* <p>
3333
* Methods that return keys in this map, like {@link #entrySet()} and {@link #keySet()}, return
34-
* keys converted to lower case.
34+
* keys converted to lower case. This map doesn't allow null key.
3535
*/
3636
@Experimental
3737
public class CaseInsensitiveStringMap implements Map<String, String> {
3838

3939
public static CaseInsensitiveStringMap empty() {
40-
return new CaseInsensitiveStringMap();
40+
return new CaseInsensitiveStringMap(new HashMap<>(0));
4141
}
4242

4343
private final Map<String, String> delegate;
4444

45-
private CaseInsensitiveStringMap() {
46-
this.delegate = new HashMap<>();
45+
public CaseInsensitiveStringMap(Map<String, String> originalMap) {
46+
this.delegate = new HashMap<>(originalMap.size());
47+
putAll(originalMap);
4748
}
4849

4950
@Override
@@ -56,9 +57,13 @@ public boolean isEmpty() {
5657
return delegate.isEmpty();
5758
}
5859

60+
private String toLowerCase(Object key) {
61+
return key.toString().toLowerCase(Locale.ROOT);
62+
}
63+
5964
@Override
6065
public boolean containsKey(Object key) {
61-
return delegate.containsKey(key.toString().toLowerCase(Locale.ROOT));
66+
return delegate.containsKey(toLowerCase(key));
6267
}
6368

6469
@Override
@@ -68,17 +73,17 @@ public boolean containsValue(Object value) {
6873

6974
@Override
7075
public String get(Object key) {
71-
return delegate.get(key.toString().toLowerCase(Locale.ROOT));
76+
return delegate.get(toLowerCase(key));
7277
}
7378

7479
@Override
7580
public String put(String key, String value) {
76-
return delegate.put(key.toLowerCase(Locale.ROOT), value);
81+
return delegate.put(toLowerCase(key), value);
7782
}
7883

7984
@Override
8085
public String remove(Object key) {
81-
return delegate.remove(key.toString().toLowerCase(Locale.ROOT));
86+
return delegate.remove(toLowerCase(key));
8287
}
8388

8489
@Override
@@ -107,4 +112,49 @@ public Collection<String> values() {
107112
public Set<Map.Entry<String, String>> entrySet() {
108113
return delegate.entrySet();
109114
}
115+
116+
/**
117+
* Returns the boolean value to which the specified key is mapped,
118+
* or defaultValue if there is no mapping for the key. The key match is case-insensitive.
119+
*/
120+
public boolean getBoolean(String key, boolean defaultValue) {
121+
String value = get(key);
122+
// We can't use `Boolean.parseBoolean` here, as it returns false for invalid strings.
123+
if (value == null) {
124+
return defaultValue;
125+
} else if (value.equalsIgnoreCase("true")) {
126+
return true;
127+
} else if (value.equalsIgnoreCase("false")) {
128+
return false;
129+
} else {
130+
throw new IllegalArgumentException(value + " is not a boolean string.");
131+
}
132+
}
133+
134+
/**
135+
* Returns the integer value to which the specified key is mapped,
136+
* or defaultValue if there is no mapping for the key. The key match is case-insensitive.
137+
*/
138+
public int getInt(String key, int defaultValue) {
139+
String value = get(key);
140+
return value == null ? defaultValue : Integer.parseInt(value);
141+
}
142+
143+
/**
144+
* Returns the long value to which the specified key is mapped,
145+
* or defaultValue if there is no mapping for the key. The key match is case-insensitive.
146+
*/
147+
public long getLong(String key, long defaultValue) {
148+
String value = get(key);
149+
return value == null ? defaultValue : Long.parseLong(value);
150+
}
151+
152+
/**
153+
* Returns the double value to which the specified key is mapped,
154+
* or defaultValue if there is no mapping for the key. The key match is case-insensitive.
155+
*/
156+
public double getDouble(String key, double defaultValue) {
157+
String value = get(key);
158+
return value == null ? defaultValue : Double.parseDouble(value);
159+
}
110160
}

sql/catalyst/src/test/java/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.java

Lines changed: 0 additions & 48 deletions
This file was deleted.

0 commit comments

Comments
 (0)