-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-40815][SQL] Add DelegateSymlinkTextInputFormat to workaround SymlinkTextInputSplit bug
#38277
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
[SPARK-40815][SQL] Add DelegateSymlinkTextInputFormat to workaround SymlinkTextInputSplit bug
#38277
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
a40e28d
fix issue
sadikovi 23c9f0f
use DelegateSymlinkTextInputFormat
sadikovi 2ff2c5e
address comments
sadikovi 814d1ca
Merge remote-tracking branch 'upstream/master' into fix-symlink-input…
sadikovi 4c0d250
trigger ci
sadikovi b65a2c2
update javadoc and address comments
sadikovi File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
121 changes: 121 additions & 0 deletions
121
sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/DelegateSymlinkTextInputFormat.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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); | ||
sunchao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| Text.writeString(out, (this.targetPath != null) ? this.targetPath.toString() : ""); | ||
| } | ||
|
|
||
| @Override | ||
| public void readFields(DataInput in) throws IOException { | ||
| super.readFields(in); | ||
sadikovi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 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); | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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 | ||
|
|
||
|
|
@@ -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") { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This test fails in JDK 11 and 17 😢 |
||
| 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] | ||
| ) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.