-
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
[SPARK-40815][SQL] Add DelegateSymlinkTextInputFormat to workaround SymlinkTextInputSplit bug
#38277
Changes from 2 commits
a40e28d
23c9f0f
2ff2c5e
814d1ca
4c0d250
b65a2c2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,111 @@ | ||
| /* | ||
| * 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. | ||
| * | ||
|
||
| * In this class, we update the split start and length to match the target file input thus fixing | ||
| * the issue. | ||
| */ | ||
| @SuppressWarnings("deprecation") | ||
|
||
| public class DelegateSymlinkTextInputFormat extends SymlinkTextInputFormat { | ||
|
|
||
| public static class DelegateSymlinkTextInputSplit extends FileSplit { | ||
| private final SymlinkTextInputSplit split; | ||
|
|
||
| public DelegateSymlinkTextInputSplit() { | ||
sunchao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| super((Path) null, 0, 0, (String[]) null); | ||
| split = new SymlinkTextInputSplit(); | ||
| } | ||
|
|
||
| public DelegateSymlinkTextInputSplit(Path symlinkPath, 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(symlinkPath, | ||
| split.getTargetSplit().getStart(), | ||
| split.getTargetSplit().getLength(), | ||
| split.getTargetSplit().getLocations()); | ||
| this.split = split; | ||
| } | ||
|
|
||
| /** | ||
| * Returns delegate input split. | ||
| */ | ||
| private SymlinkTextInputSplit getSplit() { | ||
| return split; | ||
| } | ||
|
|
||
| @Override | ||
| public void write(DataOutput out) throws IOException { | ||
| super.write(out); | ||
sunchao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| split.write(out); | ||
| } | ||
|
|
||
| @Override | ||
| public void readFields(DataInput in) throws IOException { | ||
| super.readFields(in); | ||
sadikovi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| split.readFields(in); | ||
| } | ||
| } | ||
|
|
||
| @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 { | ||
sadikovi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| InputSplit[] splits = super.getSplits(job, numSplits); | ||
| for (int i = 0; i < splits.length; i++) { | ||
| SymlinkTextInputSplit split = (SymlinkTextInputSplit) splits[i]; | ||
| splits[i] = new DelegateSymlinkTextInputSplit(split.getPath(), split); | ||
| } | ||
| return splits; | ||
| } | ||
|
|
||
| @Override | ||
| public void configure(JobConf job) { | ||
| super.configure(job); | ||
| } | ||
|
|
||
| @Override | ||
| public ContentSummary getContentSummary(Path p, JobConf job) | ||
| throws IOException { | ||
sadikovi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| return super.getContentSummary(p, job); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,12 +17,15 @@ | |
|
|
||
| package org.apache.spark.sql.hive.execution | ||
|
|
||
| import java.io.File | ||
| import java.nio.charset.StandardCharsets | ||
| import java.nio.file.Files | ||
| import java.sql.{Date, Timestamp} | ||
|
|
||
| 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.test.TestHiveSingleton | ||
| import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION | ||
| import org.apache.spark.sql.internal.SQLConf.{ORC_IMPLEMENTATION, USE_DELEGATE_FOR_SYMLINK_TEXT_INPUT_FORMAT} | ||
| import org.apache.spark.sql.test.SQLTestUtils | ||
| import org.apache.spark.tags.SlowHiveTest | ||
|
|
||
|
|
@@ -218,4 +221,48 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS | |
| checkAnswer(spark.table("t1"), Seq(Row(Array("SPARK-34512", "HIVE-24797")))) | ||
| } | ||
| } | ||
|
|
||
| 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(USE_DELEGATE_FOR_SYMLINK_TEXT_INPUT_FORMAT.key -> "false") { | ||
sadikovi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| checkAnswer( | ||
| sql("SELECT id FROM t ORDER BY id ASC"), | ||
| Seq.empty[Row] | ||
| ) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, got it. Initially, I thought this PR aims to be backported in order to fix the correctness issues of
SymlinkTextInputFormat.