Skip to content
This repository was archived by the owner on Mar 24, 2025. It is now read-only.

Commit d76a500

Browse files
authored
Changes for scala 2.13 compatibility later (#558)
1 parent 416d6b9 commit d76a500

9 files changed

Lines changed: 38 additions & 39 deletions

File tree

README.md

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -279,7 +279,7 @@ You can also use `.format("xml")` and `.load(...)`.
279279
import org.apache.spark.sql.SparkSession
280280
import com.databricks.spark.xml._
281281

282-
val spark = SparkSession.builder.getOrCreate()
282+
val spark = SparkSession.builder().getOrCreate()
283283
val df = spark.read
284284
.option("rowTag", "book")
285285
.xml("books.xml")
@@ -298,7 +298,7 @@ import org.apache.spark.sql.SparkSession
298298
import org.apache.spark.sql.types.{StructType, StructField, StringType, DoubleType}
299299
import com.databricks.spark.xml._
300300

301-
val spark = SparkSession.builder.getOrCreate()
301+
val spark = SparkSession.builder().getOrCreate()
302302
val customSchema = StructType(Array(
303303
StructField("_id", StringType, nullable = true),
304304
StructField("author", StringType, nullable = true),
@@ -372,7 +372,7 @@ df.select("author", "_id").write()
372372

373373
```python
374374
from pyspark.sql import SparkSession
375-
spark = SparkSession.builder.getOrCreate()
375+
spark = SparkSession.builder().getOrCreate()
376376

377377
df = spark.read.format('xml').options(rowTag='book').load('books.xml')
378378
df.select("author", "_id").write \
@@ -386,7 +386,7 @@ You can manually specify schema:
386386
from pyspark.sql import SparkSession
387387
from pyspark.sql.types import *
388388

389-
spark = SparkSession.builder.getOrCreate()
389+
spark = SparkSession.builder().getOrCreate()
390390
customSchema = StructType([
391391
StructField("_id", StringType(), True),
392392
StructField("author", StringType(), True),

src/main/scala/com/databricks/spark/xml/XmlReader.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,15 +32,15 @@ class XmlReader(private var schema: StructType,
3232

3333
// Explicit constructors for Java compatibility
3434

35-
def this() {
35+
def this() = {
3636
this(null, Map.empty)
3737
}
3838

39-
def this(schema: StructType) {
39+
def this(schema: StructType) = {
4040
this(schema, Map.empty)
4141
}
4242

43-
def this(options: Map[String, Any]) {
43+
def this(options: Map[String, Any]) = {
4444
this(null, options)
4545
}
4646

src/main/scala/com/databricks/spark/xml/package.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -86,7 +86,7 @@ package object xml {
8686

8787
@deprecated("Use XmlReader directly", "0.13.0")
8888
def xml(xmlDataset: Dataset[String]): DataFrame = {
89-
val spark = SparkSession.builder.getOrCreate()
89+
val spark = SparkSession.builder().getOrCreate()
9090
new XmlReader().xmlDataset(spark, xmlDataset)
9191
}
9292
}

src/main/scala/com/databricks/spark/xml/parsers/StaxXmlParser.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -122,7 +122,7 @@ private[xml] object StaxXmlParser extends Serializable {
122122
}
123123
val corruptFieldIndex = Try(schema.fieldIndex(options.columnNameOfCorruptRecord)).toOption
124124
corruptFieldIndex.foreach(resultRow(_) = record)
125-
Some(Row.fromSeq(resultRow))
125+
Some(Row.fromSeq(resultRow.toIndexedSeq))
126126
}
127127
}
128128

@@ -281,7 +281,7 @@ private[xml] object StaxXmlParser extends Serializable {
281281
// Return an empty row with all nested elements by the schema set to null.
282282
Row.fromSeq(Seq.fill(schema.fieldNames.length)(null))
283283
} else {
284-
Row.fromSeq(row)
284+
Row.fromSeq(row.toIndexedSeq)
285285
}
286286
}
287287

@@ -366,9 +366,9 @@ private[xml] object StaxXmlParser extends Serializable {
366366
}
367367

368368
if (badRecordException.isEmpty) {
369-
Row.fromSeq(row)
369+
Row.fromSeq(row.toIndexedSeq)
370370
} else {
371-
throw PartialResultException(Row.fromSeq(row), badRecordException.get)
371+
throw PartialResultException(Row.fromSeq(row.toIndexedSeq), badRecordException.get)
372372
}
373373
}
374374
}

src/main/scala/com/databricks/spark/xml/util/InferSchema.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import javax.xml.transform.stream.StreamSource
2323

2424
import scala.annotation.tailrec
2525
import scala.collection.JavaConverters._
26-
import scala.collection.Seq
2726
import scala.collection.mutable.ArrayBuffer
2827
import scala.util.control.NonFatal
2928

src/main/scala/com/databricks/spark/xml/util/XSDToSchema.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -134,7 +134,7 @@ object XSDToSchema {
134134
xmlSchema.getParent.getTypeByQName(attribute.getSchemaTypeName))
135135
StructField(s"_${attribute.getName}", baseStructField.dataType,
136136
attribute.getUse != XmlSchemaUse.REQUIRED)
137-
}
137+
}.toSeq
138138
StructField(complexType.getName, StructType(value +: attributes))
139139
case unsupported =>
140140
throw new IllegalArgumentException(s"Unsupported content: $unsupported")
@@ -153,7 +153,7 @@ object XSDToSchema {
153153
} else {
154154
StructField(element.getName, ArrayType(baseStructField.dataType), nullable)
155155
}
156-
}
156+
}.toSeq
157157
// xs:choice
158158
case choice: XmlSchemaChoice =>
159159
choice.getItems.asScala.map {
@@ -167,7 +167,7 @@ object XSDToSchema {
167167
case any: XmlSchemaAny =>
168168
val dataType = if (any.getMaxOccurs > 1) ArrayType(StringType) else StringType
169169
StructField(XmlOptions.DEFAULT_WILDCARD_COL_NAME, dataType, true)
170-
}
170+
}.toSeq
171171
// xs:sequence
172172
case sequence: XmlSchemaSequence =>
173173
// flatten xs:choice nodes
@@ -192,7 +192,7 @@ object XSDToSchema {
192192
case unsupported =>
193193
throw new IllegalArgumentException(s"Unsupported item: $unsupported")
194194
}
195-
}
195+
}.toSeq
196196
case unsupported =>
197197
throw new IllegalArgumentException(s"Unsupported particle: $unsupported")
198198
}
@@ -202,7 +202,7 @@ object XSDToSchema {
202202
xmlSchema.getParent.getTypeByQName(attribute.getSchemaTypeName))
203203
StructField(s"_${attribute.getName}", baseStructField.dataType,
204204
attribute.getUse != XmlSchemaUse.REQUIRED)
205-
}
205+
}.toSeq
206206
StructField(complexType.getName, StructType(childFields ++ attributes))
207207
case unsupported =>
208208
throw new IllegalArgumentException(s"Unsupported content model: $unsupported")

src/main/scala/com/databricks/spark/xml/util/XmlFile.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,7 @@ private[xml] object XmlFile {
105105

106106
override def hasNext: Boolean = iter.hasNext || firstRow || lastRow
107107

108-
override def next: String = {
108+
override def next(): String = {
109109
if (iter.nonEmpty) {
110110
if (firstRow) {
111111
indentingXmlWriter.writeStartElement(rootElementName)

src/test/scala/com/databricks/spark/xml/XmlSuite.scala

Lines changed: 18 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -327,8 +327,8 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
327327

328328
val carsCopy = spark.read.xml(copyFilePath.toString)
329329

330-
assert(carsCopy.count === cars.count)
331-
assert(carsCopy.collect.map(_.toString).toSet === cars.collect.map(_.toString).toSet)
330+
assert(carsCopy.count() === cars.count())
331+
assert(carsCopy.collect().map(_.toString).toSet === cars.collect().map(_.toString).toSet)
332332
}
333333

334334
test("DSL save with gzip compression codec by shorten name") {
@@ -345,8 +345,8 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
345345

346346
val carsCopy = spark.read.xml(copyFilePath.toString)
347347

348-
assert(carsCopy.count === cars.count)
349-
assert(carsCopy.collect.map(_.toString).toSet === cars.collect.map(_.toString).toSet)
348+
assert(carsCopy.count() === cars.count())
349+
assert(carsCopy.collect().map(_.toString).toSet === cars.collect().map(_.toString).toSet)
350350
}
351351

352352
test("DSL save") {
@@ -362,8 +362,8 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
362362
val booksCopy = spark.read
363363
.option("rowTag", "book")
364364
.xml(copyFilePath.toString)
365-
assert(booksCopy.count === books.count)
366-
assert(booksCopy.collect.map(_.toString).toSet === books.collect.map(_.toString).toSet)
365+
assert(booksCopy.count() === books.count())
366+
assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet)
367367
}
368368

369369
test("DSL save with nullValue and treatEmptyValuesAsNulls") {
@@ -381,8 +381,8 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
381381
.option("treatEmptyValuesAsNulls", "true")
382382
.xml(copyFilePath.toString)
383383

384-
assert(booksCopy.count === books.count)
385-
assert(booksCopy.collect.map(_.toString).toSet === books.collect.map(_.toString).toSet)
384+
assert(booksCopy.count() === books.count())
385+
assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet)
386386
}
387387

388388
test("Write values properly as given to valueTag even if it starts with attributePrefix") {
@@ -408,8 +408,8 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
408408
.option("rowTag", "book")
409409
.xml(copyFilePath.toString)
410410

411-
assert(booksCopy.count === books.count)
412-
assert(booksCopy.collect.map(_.toString).toSet === books.collect.map(_.toString).toSet)
411+
assert(booksCopy.count() === books.count())
412+
assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet)
413413
}
414414

415415
test("DSL save dataframe not read from a XML file") {
@@ -429,7 +429,7 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
429429
field("item", ArrayType(StringType))))
430430
val dfCopy = spark.read.xml(copyFilePath.toString)
431431

432-
assert(dfCopy.count === df.count)
432+
assert(dfCopy.count() === df.count())
433433
assert(dfCopy.schema === schemaCopy)
434434
}
435435

@@ -582,7 +582,7 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
582582
field("title"))
583583

584584
assert(resultsOne.schema === schemaOne)
585-
assert(resultsOne.count === 12)
585+
assert(resultsOne.count() === 12)
586586

587587
// Explicitly set
588588
val attributePrefix = "@#"
@@ -602,7 +602,7 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
602602
field("title"))
603603

604604
assert(resultsTwo.schema === schemaTwo)
605-
assert(resultsTwo.count === 12)
605+
assert(resultsTwo.count() === 12)
606606
}
607607

608608
test("DSL test schema (excluding tags) inferred correctly") {
@@ -752,7 +752,7 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
752752
assert(result(0) === Row(111, 222))
753753
}
754754

755-
private[this] def testNextedElementFromFile(xmlFile: String) = {
755+
private[this] def testNextedElementFromFile(xmlFile: String): Unit = {
756756
val lines = getLines(Paths.get(xmlFile)).toList
757757
val firstExpected = lines(2).trim
758758
val lastExpected = lines(3).trim
@@ -962,8 +962,8 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
962962
val booksCopy = spark.read
963963
.option("rowTag", "book")
964964
.xml(copyFilePath.toString)
965-
assert(booksCopy.count === books.count)
966-
assert(booksCopy.collect.map(_.toString).toSet === books.collect.map(_.toString).toSet)
965+
assert(booksCopy.count() === books.count())
966+
assert(booksCopy.collect().map(_.toString).toSet === books.collect().map(_.toString).toSet)
967967
}
968968

969969
test("DSL test nulls out invalid values when set to permissive and given explicit schema") {
@@ -1213,7 +1213,7 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
12131213
df.write.option("rootTag", "root foo='bar' bing=\"baz\"").xml(xmlPath.toString)
12141214

12151215
val xmlFile =
1216-
Files.list(xmlPath).iterator.asScala.filter(_.getFileName.toString.startsWith("part-")).next
1216+
Files.list(xmlPath).iterator.asScala.filter(_.getFileName.toString.startsWith("part-")).next()
12171217
val firstLine = getLines(xmlFile).head
12181218
assert(firstLine === "<root foo=\"bar\" bing=\"baz\">")
12191219
}
@@ -1378,7 +1378,7 @@ final class XmlSuite extends AnyFunSuite with BeforeAndAfterAll {
13781378
private def getLines(path: Path): Seq[String] = {
13791379
val source = Source.fromFile(path.toFile)
13801380
try {
1381-
source.getLines.toList
1381+
source.getLines().toList
13821382
} finally {
13831383
source.close()
13841384
}

src/test/scala/com/databricks/spark/xml/parsers/StaxXmlGeneratorSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -92,13 +92,13 @@ final class StaxXmlGeneratorSuite extends AnyFunSuite with BeforeAndAfterAll {
9292
nullDatum = null)
9393
)
9494

95-
val df = dataset.toDF.orderBy("booleanDatum")
95+
val df = dataset.toDF().orderBy("booleanDatum")
9696
val targetFile =
9797
Files.createTempDirectory("StaxXmlGeneratorSuite").resolve("roundtrip.xml").toString
9898
df.write.format("xml").save(targetFile)
9999
val newDf =
100100
spark.read.schema(df.schema).format("xml").load(targetFile).orderBy("booleanDatum")
101-
assert(df.collect.deep == newDf.collect.deep)
101+
assert(df.collect().toSeq === newDf.collect().toSeq)
102102
}
103103

104104
}

0 commit comments

Comments
 (0)