From 7f56f0b23f16fad133ffb28385808d4213c6ca54 Mon Sep 17 00:00:00 2001 From: Patrick Woody Date: Thu, 8 Feb 2018 18:48:22 -0500 Subject: [PATCH 1/2] Output valid streaming XML with woodstox --- .../databricks/spark/xml/util/XmlFile.scala | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala b/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala index 155f7edd..4b72da40 100644 --- a/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala +++ b/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala @@ -101,6 +101,10 @@ private[xml] object XmlFile { override def next: String = { if (iter.nonEmpty) { + if (firstRow) { + indentingXmlWriter.writeStartElement(options.rootTag) + firstRow = false + } val xml = { StaxXmlGenerator( rowSchema, @@ -109,21 +113,13 @@ private[xml] object XmlFile { writer.toString } writer.reset() - - // Here it needs to add indentations for the start of each line, - // in order to insert the start element and end element. - val indentedXml = indent + xml.replaceAll(rowSeparator, rowSeparator + indent) - if (firstRow) { - firstRow = false - startElement + rowSeparator + indentedXml - } else { - indentedXml - } + xml } else { - indentingXmlWriter.close() if (!firstRow) { lastRow = false - endElement + indentingXmlWriter.writeEndElement() + indentingXmlWriter.close() + writer.toString } else { // This means the iterator was initially empty. firstRow = false From 4d9284785a786e681f7c4c54eb35d8a5f81391ef Mon Sep 17 00:00:00 2001 From: Patrick Woody Date: Sat, 10 Feb 2018 16:01:09 -0500 Subject: [PATCH 2/2] Removed unused variables --- src/main/scala/com/databricks/spark/xml/util/XmlFile.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala b/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala index 4b72da40..81f915d2 100644 --- a/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala +++ b/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala @@ -32,7 +32,6 @@ import com.databricks.spark.xml.{XmlOptions, XmlInputFormat} private[xml] object XmlFile { val DEFAULT_INDENT = " " - val DEFAULT_ROW_SEPARATOR = "\n" def withCharset( context: SparkContext, @@ -80,11 +79,8 @@ private[xml] object XmlFile { parameters: Map[String, String] = Map()): Unit = { val options = XmlOptions(parameters.toMap) val codecClass = CompressionCodecs.getCodecClass(options.codec) - val startElement = s"<${options.rootTag}>" - val endElement = s"" val rowSchema = dataFrame.schema val indent = XmlFile.DEFAULT_INDENT - val rowSeparator = XmlFile.DEFAULT_ROW_SEPARATOR val xmlRDD = dataFrame.rdd.mapPartitions { iter => val factory = XMLOutputFactory.newInstance()