Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.hadoop.hive.ql.io;

import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;

import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;

/**
* Delegate for SymlinkTextInputFormat, created to address SPARK-40815.
* Fixes an issue where SymlinkTextInputFormat returns empty splits which could result in
* the correctness issue when "spark.hadoopRDD.ignoreEmptySplits" is enabled.
* <p>
* In this class, we update the split start and length to match the target file input thus fixing
* the issue.
*/
public class DelegateSymlinkTextInputFormat extends SymlinkTextInputFormat {

public static class DelegateSymlinkTextInputSplit extends FileSplit {
private Path targetPath; // Path to the actual data file, not the symlink file.

// Used for deserialisation.
public DelegateSymlinkTextInputSplit() {
super((Path) null, 0, 0, (String[]) null);
targetPath = null;
}

public DelegateSymlinkTextInputSplit(SymlinkTextInputSplit split) throws IOException {
// It is fine to set start and length to the target file split because
// SymlinkTextInputFormat maintains 1-1 mapping between SymlinkTextInputSplit and FileSplit.
super(split.getPath(),
split.getTargetSplit().getStart(),
split.getTargetSplit().getLength(),
split.getTargetSplit().getLocations());
this.targetPath = split.getTargetSplit().getPath();
}

/**
* Returns target path.
* Visible for testing.
*/
public Path getTargetPath() {
return targetPath;
}

/**
* Reconstructs the delegate input split.
*/
private SymlinkTextInputSplit getSplit() throws IOException {
return new SymlinkTextInputSplit(
getPath(),
new FileSplit(targetPath, getStart(), getLength(), getLocations())
);
}

@Override
public void write(DataOutput out) throws IOException {
super.write(out);
Text.writeString(out, (this.targetPath != null) ? this.targetPath.toString() : "");
}

@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
String target = Text.readString(in);
this.targetPath = (!target.isEmpty()) ? new Path(target) : null;
}
}

@Override
public RecordReader<LongWritable, Text> getRecordReader(
InputSplit split, JobConf job, Reporter reporter) throws IOException {
InputSplit targetSplit = ((DelegateSymlinkTextInputSplit) split).getSplit();
return super.getRecordReader(targetSplit, job, reporter);
}

@Override
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
InputSplit[] splits = super.getSplits(job, numSplits);
for (int i = 0; i < splits.length; i++) {
SymlinkTextInputSplit split = (SymlinkTextInputSplit) splits[i];
splits[i] = new DelegateSymlinkTextInputSplit(split);
}
return splits;
}

@Override
public void configure(JobConf job) {
super.configure(job);
}

@Override
public ContentSummary getContentSummary(Path p, JobConf job) throws IOException {
return super.getContentSummary(p, job);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,15 @@ private[spark] object HiveUtils extends Logging {
.booleanConf
.createWithDefault(true)

val USE_DELEGATE_FOR_SYMLINK_TEXT_INPUT_FORMAT =
buildConf("spark.sql.hive.useDelegateForSymlinkTextInputFormat")
.internal()
.doc("When true, SymlinkTextInputFormat is replaced with a similar delegate class during " +
"table scan in order to fix the issue of empty splits")
.version("3.4.0")
.booleanConf
.createWithDefault(true)

/**
* The version of the hive client that will be used to communicate with the metastore. Note that
* this does not necessarily need to be the same version of Hive that is used internally by
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,14 @@ package org.apache.spark.sql.hive.execution
import scala.collection.JavaConverters._

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hive.ql.io.{DelegateSymlinkTextInputFormat, SymlinkTextInputFormat}
import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition}
import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.hadoop.hive.serde.serdeConstants
import org.apache.hadoop.hive.serde2.objectinspector._
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
import org.apache.hadoop.mapred.InputFormat

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
Expand Down Expand Up @@ -89,7 +91,7 @@ case class HiveTableScanExec(

@transient private lazy val hiveQlTable = HiveClientImpl.toHiveTable(relation.tableMeta)
@transient private lazy val tableDesc = new TableDesc(
hiveQlTable.getInputFormatClass,
getInputFormat(hiveQlTable.getInputFormatClass, conf),
hiveQlTable.getOutputFormatClass,
hiveQlTable.getMetadata)

Expand Down Expand Up @@ -231,6 +233,20 @@ case class HiveTableScanExec(
predicates.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral))
}

// Optionally returns a delegate input format based on the provided input format class.
// This is currently used to replace SymlinkTextInputFormat with DelegateSymlinkTextInputFormat
// in order to fix SPARK-40815.
private def getInputFormat(
inputFormatClass: Class[_ <: InputFormat[_, _]],
conf: SQLConf): Class[_ <: InputFormat[_, _]] = {
if (inputFormatClass == classOf[SymlinkTextInputFormat] &&
conf != null && conf.getConf(HiveUtils.USE_DELEGATE_FOR_SYMLINK_TEXT_INPUT_FORMAT)) {
classOf[DelegateSymlinkTextInputFormat]
} else {
inputFormatClass
}
}

override def doCanonicalize(): HiveTableScanExec = {
val input: AttributeSeq = relation.output
HiveTableScanExec(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,19 @@

package org.apache.spark.sql.hive.execution

import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream, File}
import java.nio.charset.StandardCharsets
import java.nio.file.Files
import java.sql.{Date, Timestamp}

import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.ql.io.{DelegateSymlinkTextInputFormat, SymlinkTextInputFormat}
import org.apache.hadoop.mapred.FileSplit;
import org.apache.spark.internal.config.HADOOP_RDD_IGNORE_EMPTY_SPLITS
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.hive.HiveUtils.{CONVERT_METASTORE_ORC, CONVERT_METASTORE_PARQUET}
import org.apache.spark.sql.hive.HiveUtils.{CONVERT_METASTORE_ORC, CONVERT_METASTORE_PARQUET, USE_DELEGATE_FOR_SYMLINK_TEXT_INPUT_FORMAT}
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION
import org.apache.spark.sql.internal.SQLConf.{ORC_IMPLEMENTATION}
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.tags.SlowHiveTest

Expand Down Expand Up @@ -218,4 +225,86 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS
checkAnswer(spark.table("t1"), Seq(Row(Array("SPARK-34512", "HIVE-24797"))))
}
}

test("SPARK-40815: DelegateSymlinkTextInputFormat serialization") {
def assertSerDe(split: DelegateSymlinkTextInputFormat.DelegateSymlinkTextInputSplit): Unit = {
val buf = new ByteArrayOutputStream()
val out = new DataOutputStream(buf)
try {
split.write(out)
} finally {
out.close()
}

val res = new DelegateSymlinkTextInputFormat.DelegateSymlinkTextInputSplit()
val in = new DataInputStream(new ByteArrayInputStream(buf.toByteArray()))
try {
res.readFields(in)
} finally {
in.close()
}

assert(split.getPath == res.getPath)
assert(split.getStart == res.getStart)
assert(split.getLength == res.getLength)
assert(split.getLocations.toSeq == res.getLocations.toSeq)
assert(split.getTargetPath == res.getTargetPath)
}

assertSerDe(
new DelegateSymlinkTextInputFormat.DelegateSymlinkTextInputSplit(
new SymlinkTextInputFormat.SymlinkTextInputSplit(
new Path("file:/tmp/symlink"),
new FileSplit(new Path("file:/tmp/file"), 1L, 2L, Array[String]())
)
)
)
}

test("SPARK-40815: Read SymlinkTextInputFormat") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test fails in JDK 11 and 17 😢
https://github.com/apache/spark/actions/runs/3379157338/jobs/5610899432
https://github.com/apache/spark/actions/runs/3381461270/jobs/5615405153

[info] - SPARK-40815: Read SymlinkTextInputFormat *** FAILED *** (587 milliseconds)
[info]   Results do not match for query:
[info]   Timezone: sun.util.calendar.ZoneInfo[id="America/Los_Angeles",offset=-28800000,dstSavings=3600000,useDaylight=true,transitions=185,lastRule=java.util.SimpleTimeZone[id=America/Los_Angeles,offset=-28800000,dstSavings=3600000,useDaylight=true,startYear=0,startMode=3,startMonth=2,startDay=8,startDayOfWeek=1,startTime=7200000,startTimeMode=0,endMode=3,endMonth=10,endDay=1,endDayOfWeek=1,endTime=7200000,endTimeMode=0]]
[info]   Timezone Env: 
[info]   
[info]   == Parsed Logical Plan ==
[info]   'Sort ['id ASC NULLS FIRST], true
[info]   +- 'Project ['id]
[info]      +- 'UnresolvedRelation [t], [], false
[info]   
[info]   == Analyzed Logical Plan ==
[info]   id: bigint
[info]   Sort [id#175602L ASC NULLS FIRST], true
[info]   +- Project [id#175602L]
[info]      +- SubqueryAlias spark_catalog.default.t
[info]         +- HiveTableRelation [`spark_catalog`.`default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [id#175602L], Partition Cols: []]
[info]   
[info]   == Optimized Logical Plan ==
[info]   Sort [id#175602L ASC NULLS FIRST], true
[info]   +- HiveTableRelation [`spark_catalog`.`default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [id#175602L], Partition Cols: []]
[info]   
[info]   == Physical Plan ==
[info]   AdaptiveSparkPlan isFinalPlan=true
[info]   +- == Final Plan ==
[info]      LocalTableScan <empty>, [id#175602L]
[info]   +- == Initial Plan ==
[info]      Sort [id#175602L ASC NULLS FIRST], true, 0
[info]      +- Exchange rangepartitioning(id#175602L ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=176717]
[info]         +- Scan hive spark_catalog.default.t [id#175602L], HiveTableRelation [`spark_catalog`.`default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [id#175602L], Partition Cols: []]
[info]   
[info]   == Results ==
[info]   
[info]   == Results ==
[info]   !== Correct Answer - 10 ==   == Spark Answer - 0 ==
[info]    struct<>                    struct<>
[info]   ![0]                         
[info]   ![1]                         
[info]   ![2]                         
[info]   ![3]                         
[info]   ![4]                         
[info]   ![5]                         
[info]   ![6]                         
[info]   ![7]                         
[info]   ![8]                         
[info]   ![9] (QueryTest.scala:243)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info]   at org.apache.spark.sql.QueryTest$.newAssertionFailedException(QueryTest.scala:233)
[info]   at org.scalatest.Assertions.fail(Assertions.scala:933)
[info]   at org.scalatest.Assertions.fail$(Assertions.scala:929)
[info]   at org.apache.spark.sql.QueryTest$.fail(QueryTest.scala:233)
[info]   at org.apache.spark.sql.QueryTest$.checkAnswer(QueryTest.scala:243)
[info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:150)
[info]   at org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite.$anonfun$new$9(HiveSerDeReadWriteSuite.scala:293)
[info]   at org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite.$anonfun$new$9$adapted(HiveSerDeReadWriteSuite.scala:266)
[info]   at org.apache.spark.sql.test.SQLTestUtils.$anonfun$withTempDir$1(SQLTestUtils.scala:79)
[info]   at org.apache.spark.sql.test.SQLTestUtils.$anonfun$withTempDir$1$adapted(SQLTestUtils.scala:78)
[info]   at org.apache.spark.SparkFunSuite.withTempDir(SparkFunSuite.scala:225)
[info]   at org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite.org$apache$spark$sql$test$SQLTestUtils$$super$withTempDir(HiveSerDeReadWriteSuite.scala:37)
[info]   at org.apache.spark.sql.test.SQLTestUtils.withTempDir(SQLTestUtils.scala:78)
[info]   at org.apache.spark.sql.test.SQLTestUtils.withTempDir$(SQLTestUtils.scala:77)
[info]   at org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite.withTempDir(HiveSerDeReadWriteSuite.scala:37)
[info]   at org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite.$anonfun$new$8(HiveSerDeReadWriteSuite.scala:266)
[info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1491)
[info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withTable(SQLTestUtils.scala:306)
[info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withTable$(SQLTestUtils.scala:304)
[info]   at org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite.withTable(HiveSerDeReadWriteSuite.scala:37)
[info]   at org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite.$anonfun$new$7(HiveSerDeReadWriteSuite.scala:266)
[info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
[info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
[info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
[info]   at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:207)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
[info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
[info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:66)

withTable("t") {
withTempDir { root =>
val dataPath = new File(root, "data")
val symlinkPath = new File(root, "symlink")

spark.range(10).selectExpr("cast(id as string) as value")
.repartition(4).write.text(dataPath.getAbsolutePath)

// Generate symlink manifest file.
val files = dataPath.listFiles().filter(_.getName.endsWith(".txt"))
assert(files.length > 0)

symlinkPath.mkdir()
Files.write(
new File(symlinkPath, "symlink.txt").toPath,
files.mkString("\n").getBytes(StandardCharsets.UTF_8)
)

sql(s"""
CREATE TABLE t (id bigint)
STORED AS
INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat'
OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
LOCATION '${symlinkPath.getAbsolutePath}';
""")

checkAnswer(
sql("SELECT id FROM t ORDER BY id ASC"),
(0 until 10).map(Row(_))
)

// Verify that with the flag disabled, we use the original SymlinkTextInputFormat
// which has the empty splits issue and therefore the result should be empty.
withSQLConf(
HADOOP_RDD_IGNORE_EMPTY_SPLITS.key -> "true",
USE_DELEGATE_FOR_SYMLINK_TEXT_INPUT_FORMAT.key -> "false") {

checkAnswer(
sql("SELECT id FROM t ORDER BY id ASC"),
Seq.empty[Row]
)
}
}
}
}
}