Skip to content

Commit

Permalink
fix: Fix 3.5.0 compile issues
Browse files Browse the repository at this point in the history
  • Loading branch information
nightscape committed Nov 14, 2023
1 parent cda2140 commit fa067f7
Show file tree
Hide file tree
Showing 7 changed files with 211 additions and 6 deletions.
20 changes: 18 additions & 2 deletions build.sc
Original file line number Diff line number Diff line change
Expand Up @@ -12,8 +12,24 @@ trait SparkModule extends Cross.Module2[String, String] with SbtModule with CiRe
override def millSourcePath = super.millSourcePath / os.up

// Custom source layout for Spark Data Source API 2
val sparkVersionSpecificSources = if (sparkVersion >= "3.4.0") {
Seq("scala", "3.0_and_up/scala", "3.1_and_up/scala", "3.2_and_up/scala", "3.3_and_up/scala", "3.4_and_up/scala")
val sparkVersionSpecificSources = if (sparkVersion >= "3.5.0") {
Seq(
"scala",
"3.0_and_up/scala",
"3.1_and_up/scala",
"3.2_and_up/scala",
"3.3_and_up/scala",
"3.5_and_up/scala"
)
} else if (sparkVersion >= "3.4.0") {
Seq(
"scala",
"3.0_and_up/scala",
"3.1_and_up/scala",
"3.2_and_up/scala",
"3.3_and_up/scala",
"3.4_and_up/scala"
)
} else if (sparkVersion >= "3.3.0") {
Seq(
"scala",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Copyright 2022 Martin Mauch (@nightscape)
*
* Licensed 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 com.crealytics.spark.excel.v2

import _root_.org.apache.spark.sql.catalyst.util.BadRecordException
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.sql.catalyst.InternalRow

trait ExcelParserBase {

protected def getCurrentInput: UTF8String
def badRecord(partialResults: Array[InternalRow], baseException: Throwable): BadRecordException =
BadRecordException(() => getCurrentInput, () => partialResults.headOption, baseException)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Copyright 2022 Martin Mauch (@nightscape)
*
* Licensed 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 com.crealytics.spark.excel.v2

import _root_.org.apache.spark.sql.catalyst.util.BadRecordException
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.sql.catalyst.InternalRow

trait ExcelParserBase {

protected def getCurrentInput: UTF8String
def badRecord(partialResults: Array[InternalRow], baseException: Throwable): BadRecordException =
BadRecordException(() => getCurrentInput, () => partialResults.headOption, baseException)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Copyright 2022 Martin Mauch (@nightscape)
*
* Licensed 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 com.crealytics.spark.excel.v2

import _root_.org.apache.spark.sql.catalyst.util.BadRecordException
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.sql.catalyst.InternalRow

trait ExcelParserBase {

protected def getCurrentInput: UTF8String
def badRecord(partialResults: Array[InternalRow], baseException: Throwable): BadRecordException =
BadRecordException(() => getCurrentInput, () => partialResults, baseException)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* Copyright 2022 Martin Mauch (@nightscape)
*
* Licensed 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 com.crealytics.spark.excel.v2

import org.apache.hadoop.conf.Configuration
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.sql.catalyst.{InternalRow, FileSourceOptions}
import org.apache.spark.sql.connector.read.PartitionReader
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.execution.datasources.v2._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.SerializableConfiguration

import java.net.URI
import scala.util.control.NonFatal

/** A factory used to create Excel readers.
*
* @param sqlConf
* SQL configuration.
* @param broadcastedConf
* Broadcasted serializable Hadoop Configuration.
* @param dataSchema
* Schema of Excel files.
* @param readDataSchema
* Required data schema in the batch scan.
* @param partitionSchema
* Schema of partitions.
* @param parsedOptions
* Options for parsing Excel files.
*/
case class ExcelPartitionReaderFactory(
sqlConf: SQLConf,
broadcastedConf: Broadcast[SerializableConfiguration],
dataSchema: StructType,
readDataSchema: StructType,
partitionSchema: StructType,
parsedOptions: ExcelOptions,
filters: Seq[Filter]
) extends FilePartitionReaderFactory {
protected def options: FileSourceOptions = new FileSourceOptions(Map(
FileSourceOptions.IGNORE_CORRUPT_FILES -> "true",
FileSourceOptions.IGNORE_MISSING_FILES -> "true"
))
override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = {
val conf = broadcastedConf.value.value
val actualDataSchema =
StructType(dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord))
val actualReadDataSchema =
StructType(readDataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord))
val parser = new ExcelParser(actualDataSchema, actualReadDataSchema, parsedOptions, filters)
val headerChecker =
new ExcelHeaderChecker(actualReadDataSchema, parsedOptions, source = s"Excel file: ${file.filePath}")
val iter = readFile(conf, file, parser, headerChecker, readDataSchema)
val partitionReader = new SparkExcelPartitionReaderFromIterator(iter)
new PartitionReaderWithPartitionValues(partitionReader, readDataSchema, partitionSchema, file.partitionValues)
}

private def readFile(
conf: Configuration,
file: PartitionedFile,
parser: ExcelParser,
headerChecker: ExcelHeaderChecker,
requiredSchema: StructType
): SheetData[InternalRow] = {
val excelHelper = ExcelHelper(parsedOptions)
val sheetData = excelHelper.getSheetData(conf, URI.create(file.filePath.toString))
try {
SheetData(
ExcelParser.parseIterator(sheetData.rowIterator, parser, headerChecker, requiredSchema),
sheetData.resourcesToClose
)
} catch {
case NonFatal(t) => {
sheetData.close()
throw t
}
}
}

}

private class SparkExcelPartitionReaderFromIterator(sheetData: SheetData[InternalRow])
extends PartitionReaderFromIterator[InternalRow](sheetData.rowIterator) {
override def close(): Unit = {
super.close()
sheetData.close()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ import org.apache.poi.ss.usermodel.DateUtil
* The pushdown filters that should be applied to converted values.
*/
class ExcelParser(dataSchema: StructType, requiredSchema: StructType, val options: ExcelOptions, filters: Seq[Filter])
extends Logging {
extends Logging with ExcelParserBase {
require(
requiredSchema.toSet.subsetOf(dataSchema.toSet),
s"requiredSchema (${requiredSchema.catalogString}) should be the subset of " +
Expand Down Expand Up @@ -95,7 +95,7 @@ class ExcelParser(dataSchema: StructType, requiredSchema: StructType, val option
private val pushedFilters = new ExcelFilters(filters, requiredSchema)

/* Retrieve the raw record string. */
private def getCurrentInput: UTF8String = UTF8String
protected def getCurrentInput: UTF8String = UTF8String
.fromString("TODO: how to show the corrupted record?")

/** This parser first picks some tokens from the input tokens, according to the required schema, then parse these
Expand Down Expand Up @@ -352,7 +352,7 @@ class ExcelParser(dataSchema: StructType, requiredSchema: StructType, val option

private def convert(tokens: Vector[Cell]): Option[InternalRow] = {
if (tokens == null) {
throw BadRecordException(() => getCurrentInput, () => None, new RuntimeException("Malformed Excel record"))
throw badRecord(Array.empty, new RuntimeException("Malformed Excel record"))
}

var badRecordException: Option[Throwable] =
Expand Down Expand Up @@ -396,7 +396,7 @@ class ExcelParser(dataSchema: StructType, requiredSchema: StructType, val option
if (skipRow) { noRows }
else {
if (badRecordException.isDefined) {
throw BadRecordException(() => getCurrentInput, () => requiredRow.headOption, badRecordException.get)
throw badRecord(requiredRow.toArray, badRecordException.get)
} else { requiredRow }
}
}
Expand Down

0 comments on commit fa067f7

Please sign in to comment.