Skip to content

Commit

Permalink
Fix potential Kerberos failure with SymlinkTextInputFormat
Browse files Browse the repository at this point in the history
  • Loading branch information
akshayrai authored Mar 29, 2023
1 parent d85df0d commit daa80f6
Showing 1 changed file with 5 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -55,14 +55,14 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.JobConfigurable;
import org.apache.hadoop.mapred.TextInputFormat;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.util.StringUtils;

import java.io.BufferedReader;
import java.io.IOException;
Expand Down Expand Up @@ -155,6 +155,8 @@ public class BackgroundHiveSplitLoader

private static final ListenableFuture<Void> COMPLETED_FUTURE = immediateVoidFuture();

private static final String FILE_INPUT_FORMAT_INPUT_DIR = "mapreduce.input.fileinputformat.inputdir";

private final Table table;
private final TupleDomain<? extends ColumnHandle> compactEffectivePredicate;
private final DynamicFilter dynamicFilter;
Expand Down Expand Up @@ -529,7 +531,7 @@ private ListenableFuture<Void> loadPartition(HivePartitionMetadata partition)
}

JobConf jobConf = toJobConf(configuration);
FileInputFormat.setInputPaths(jobConf, path);
jobConf.set(FILE_INPUT_FORMAT_INPUT_DIR, StringUtils.escapeString(path.toString()));
// Pass SerDes and Table parameters into input format configuration
fromProperties(schema).forEach(jobConf::set);
InputSplit[] splits = hdfsEnvironment.doAs(hdfsContext.getIdentity(), () -> inputFormat.getSplits(jobConf, 0));
Expand Down Expand Up @@ -591,7 +593,7 @@ private ListenableFuture<Void> createHiveSymlinkSplits(
if (targetInputFormat instanceof JobConfigurable) {
((JobConfigurable) targetInputFormat).configure(targetJob);
}
FileInputFormat.setInputPaths(targetJob, targetPath);
targetJob.set(FILE_INPUT_FORMAT_INPUT_DIR, StringUtils.escapeString(targetPath.toString()));
InputSplit[] targetSplits = hdfsEnvironment.doAs(
hdfsContext.getIdentity(),
() -> targetInputFormat.getSplits(targetJob, 0));
Expand Down

0 comments on commit daa80f6

Please sign in to comment.