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
Expand Up @@ -30,6 +30,7 @@
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.TextInputFormat;

/**
* Delegate for SymlinkTextInputFormat, created to address SPARK-40815.
Expand Down Expand Up @@ -95,8 +96,18 @@ public void readFields(DataInput in) throws IOException {
@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);
DelegateSymlinkTextInputSplit delegateSplit = (DelegateSymlinkTextInputSplit) split;
InputSplit targetSplit = ((SymlinkTextInputSplit) delegateSplit.getSplit()).getTargetSplit();

// SPARK-40815: the code is derived from
// https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java
// However, we use the TextInputFormat record reader directly without HiveRecordReader to avoid
// ExecMapper.getDone() checks.

// The target data is in TextInputFormat.
TextInputFormat inputFormat = new TextInputFormat();
inputFormat.configure(job);
return inputFormat.getRecordReader(targetSplit, job, reporter);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import java.nio.charset.StandardCharsets
import java.nio.file.Files
import java.sql.{Date, Timestamp}

import org.apache.commons.lang3.{JavaVersion, SystemUtils}
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.ql.io.{DelegateSymlinkTextInputFormat, SymlinkTextInputFormat}
import org.apache.hadoop.mapred.FileSplit;
Expand Down Expand Up @@ -228,9 +227,6 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS
}

test("SPARK-40815: DelegateSymlinkTextInputFormat serialization") {
// Ignored due to JDK 11 failures reported in https://github.com/apache/spark/pull/38277.
assume(!SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9))

def assertSerDe(split: DelegateSymlinkTextInputFormat.DelegateSymlinkTextInputSplit): Unit = {
val buf = new ByteArrayOutputStream()
val out = new DataOutputStream(buf)
Expand Down Expand Up @@ -266,9 +262,6 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS
}

test("SPARK-40815: Read SymlinkTextInputFormat") {
// Ignored due to JDK 11 failures reported in https://github.com/apache/spark/pull/38277.
assume(!SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9))

withTable("t") {
withTempDir { root =>
val dataPath = new File(root, "data")
Expand Down Expand Up @@ -300,6 +293,12 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS
(0 until 10).map(Row(_))
)

// Verify limit since we bypass ExecMapper.getDone().
checkAnswer(
sql("SELECT id FROM t ORDER BY id ASC LIMIT 2"),
(0 until 2).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(
Expand Down