diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index eba7aa386ade..9e1f8698480e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -70,7 +70,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable val configuration = job.getConfiguration - configuration.set(OrcRelation.ORC_COMPRESSION, orcOptions.compressionCodec) + configuration.set(OrcFileFormat.ORC_COMPRESSION, orcOptions.compressionCodec) configuration match { case conf: JobConf => conf.setOutputFormat(classOf[OrcOutputFormat]) @@ -110,7 +110,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable if (sparkSession.sessionState.conf.orcFilterPushDown) { // Sets pushed predicates OrcFilters.createFilter(requiredSchema, filters.toArray).foreach { f => - hadoopConf.set(OrcRelation.SARG_PUSHDOWN, f.toKryo) + hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, f.toKryo) hadoopConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true) } } @@ -129,7 +129,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable Iterator.empty } else { val physicalSchema = maybePhysicalSchema.get - OrcRelation.setRequiredColumns(conf, physicalSchema, requiredSchema) + OrcFileFormat.setRequiredColumns(conf, physicalSchema, requiredSchema) val orcRecordReader = { val job = Job.getInstance(conf) @@ -151,7 +151,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => recordsIterator.close())) // Unwraps `OrcStruct`s to `UnsafeRow`s - OrcRelation.unwrapOrcStructs( + OrcFileFormat.unwrapOrcStructs( conf, requiredSchema, Some(orcRecordReader.getObjectInspector.asInstanceOf[StructObjectInspector]), @@ -218,8 +218,8 @@ private[orc] class OrcOutputWriter( override val path: String = { val compressionExtension: String = { - val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION) - OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") + val name = context.getConfiguration.get(OrcFileFormat.ORC_COMPRESSION) + OrcFileFormat.extensionsForCompressionCodecNames.getOrElse(name, "") } // It has the `.orc` extension at the end because (de)compression tools // such as gunzip would not be able to decompress this as the compression @@ -257,7 +257,7 @@ private[orc] class OrcOutputWriter( } } -private[orc] object OrcRelation extends HiveInspectors { +private[orc] object OrcFileFormat extends HiveInspectors { // The references of Hive's classes will be minimized. val ORC_COMPRESSION = "orc.compress" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala index c2a126d3bf9c..0d6c63a2d5ed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala @@ -33,7 +33,7 @@ private[orc] class OrcOptions(@transient private val parameters: Map[String, Str // `orc.compress` is a ORC configuration. So, here we respect this as an option but // `compression` has higher precedence than `orc.compress`. It means if both are set, // we will use `compression`. - val orcCompressionConf = parameters.get(OrcRelation.ORC_COMPRESSION) + val orcCompressionConf = parameters.get(OrcFileFormat.ORC_COMPRESSION) val codecName = parameters .get("compression") .orElse(orcCompressionConf)