You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2021/01/29 17:58:19 UTC
[iceberg] branch master updated: Spark: Do not modify location in
HadoopInputFile (#2170)
This is an automated email from the ASF dual-hosted git repository.
blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git
The following commit(s) were added to refs/heads/master by this push:
new f63fa5c Spark: Do not modify location in HadoopInputFile (#2170)
f63fa5c is described below
commit f63fa5c44d03fb6c830bb209bd2cc7e19df04ee9
Author: Wing Yew Poon <wy...@gmail.com>
AuthorDate: Fri Jan 29 09:58:02 2021 -0800
Spark: Do not modify location in HadoopInputFile (#2170)
---
.../org/apache/iceberg/hadoop/HadoopInputFile.java | 40 ++++++++++++++++------
1 file changed, 29 insertions(+), 11 deletions(-)
diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java
index 39a6d82..7cf6dee 100644
--- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java
+++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java
@@ -43,6 +43,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists;
public class HadoopInputFile implements InputFile {
public static final String[] NO_LOCATION_PREFERENCE = new String[0];
+ private final String location;
private final FileSystem fs;
private final Path path;
private final Configuration conf;
@@ -50,25 +51,23 @@ public class HadoopInputFile implements InputFile {
private Long length = null;
public static HadoopInputFile fromLocation(CharSequence location, Configuration conf) {
- Path path = new Path(location.toString());
- return fromPath(path, conf);
+ FileSystem fs = Util.getFs(new Path(location.toString()), conf);
+ return new HadoopInputFile(fs, location.toString(), conf);
}
public static HadoopInputFile fromLocation(CharSequence location, long length,
Configuration conf) {
- Path path = new Path(location.toString());
- return fromPath(path, length, conf);
+ FileSystem fs = Util.getFs(new Path(location.toString()), conf);
+ return new HadoopInputFile(fs, location.toString(), length, conf);
}
public static HadoopInputFile fromLocation(CharSequence location, FileSystem fs) {
- Path path = new Path(location.toString());
- return fromPath(path, fs);
+ return new HadoopInputFile(fs, location.toString(), fs.getConf());
}
public static HadoopInputFile fromLocation(CharSequence location, long length,
FileSystem fs) {
- Path path = new Path(location.toString());
- return fromPath(path, length, fs);
+ return new HadoopInputFile(fs, location.toString(), length, fs.getConf());
}
public static HadoopInputFile fromPath(Path path, Configuration conf) {
@@ -110,9 +109,26 @@ public class HadoopInputFile implements InputFile {
return new HadoopInputFile(fs, stat, conf);
}
+ private HadoopInputFile(FileSystem fs, String location, Configuration conf) {
+ this.fs = fs;
+ this.location = location;
+ this.path = new Path(location);
+ this.conf = conf;
+ }
+
+ private HadoopInputFile(FileSystem fs, String location, long length, Configuration conf) {
+ Preconditions.checkArgument(length >= 0, "Invalid file length: %s", length);
+ this.fs = fs;
+ this.location = location;
+ this.path = new Path(location);
+ this.conf = conf;
+ this.length = length;
+ }
+
private HadoopInputFile(FileSystem fs, Path path, Configuration conf) {
this.fs = fs;
this.path = path;
+ this.location = path.toString();
this.conf = conf;
}
@@ -120,6 +136,7 @@ public class HadoopInputFile implements InputFile {
Preconditions.checkArgument(length >= 0, "Invalid file length: %s", length);
this.fs = fs;
this.path = path;
+ this.location = path.toString();
this.conf = conf;
this.length = length;
}
@@ -127,6 +144,7 @@ public class HadoopInputFile implements InputFile {
private HadoopInputFile(FileSystem fs, FileStatus stat, Configuration conf) {
this.fs = fs;
this.path = stat.getPath();
+ this.location = path.toString();
this.stat = stat;
this.conf = conf;
this.length = stat.getLen();
@@ -181,8 +199,8 @@ public class HadoopInputFile implements InputFile {
public String[] getBlockLocations(long start, long end) {
List<String> hosts = Lists.newArrayList();
try {
- for (BlockLocation location : fs.getFileBlockLocations(path, start, end)) {
- Collections.addAll(hosts, location.getHosts());
+ for (BlockLocation bl : fs.getFileBlockLocations(path, start, end)) {
+ Collections.addAll(hosts, bl.getHosts());
}
return hosts.toArray(NO_LOCATION_PREFERENCE);
@@ -194,7 +212,7 @@ public class HadoopInputFile implements InputFile {
@Override
public String location() {
- return path.toString();
+ return location;
}
@Override