Skip to content
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

Add "open" method #1895

Merged
merged 3 commits into from
May 9, 2021
Merged
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
86 changes: 33 additions & 53 deletions clients/hadoopfs/src/main/java/io/lakefs/LakeFSFileSystem.java
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,10 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Progressable;

import io.lakefs.clients.api.ApiClient;
import io.lakefs.clients.api.ObjectsApi;
import io.lakefs.clients.api.model.ObjectStats;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
Expand Down Expand Up @@ -43,15 +47,27 @@ public class LakeFSFileSystem extends FileSystem {
private static final String BASIC_AUTH = "basic_auth";
private static final String SEPARATOR = "/";

private Configuration conf;
private URI uri;
private Path workingDirectory = new Path(SEPARATOR);
private ApiClient apiClient;

private URI translateUri(URI uri) throws java.net.URISyntaxException {
switch (uri.getScheme()) {
case "s3":
return new URI("s3a", uri.getUserInfo(), uri.getHost(), uri.getPort(), uri.getPath(), uri.getQuery(),
uri.getFragment());
default:
throw new RuntimeException(String.format("unsupported URI scheme %s", uri.getScheme()));
}
}

public URI getUri() { return uri; }

@Override
public void initialize(URI name, Configuration conf) throws IOException {
super.initialize(name, conf);
this.conf = conf;
LOG.debug("$$$$$$$$$$$$$$$$$$$$$$$$$$$$ initialize: {} $$$$$$$$$$$$$$$$$$$$$$$$$$$$", name);

String host = name.getHost();
Expand Down Expand Up @@ -85,11 +101,23 @@ public void initialize(URI name, Configuration conf) throws IOException {
* regardless of the given file path.
*/
@Override
public FSDataInputStream open(Path path, int i) throws IOException {
LOG.debug("$$$$$$$$$$$$$$$$$$$$$$$$$$$$ Calling open method for: {} $$$$$$$$$$$$$$$$$$$$$$$$$$$$", path.getName());
String strToWrite = "abc";
MyInputStream inputStream = new MyInputStream(strToWrite);
return new FSDataInputStream(inputStream);
public FSDataInputStream open(Path path, int bufSize) throws IOException {
try {
LOG.debug("$$$$$$$$$$$$$$$$$$$$$$$$$$$$ open(" + path.getName() + ") $$$$$$$$$$$$$$$$$$$$$$$$$$$$");

ObjectsApi objects = new ObjectsApi(apiClient);
ObjectLocation objLoc = pathToObjectLocation(path);
ObjectStats stats = objects.statObject(objLoc.getRepository(), objLoc.getRef(), objLoc.getPath());
URI physicalUri = translateUri(new URI(stats.getPhysicalAddress()));

Path physicalPath = new Path(physicalUri.toString());
FileSystem physicalFs = physicalPath.getFileSystem(conf);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the part where I thought we need to initialize the underlying filesystem once in the initialize and use it here and just call open.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I was hoping to do that too. Disappointed that this is actually surprisingly difficult to do correctly. Now according to S3AFileSystem.initialize (please look at the code), the repository is guaranteed to exist on the URL. (TBH, I don't understand why, but we might assume as much). So we could generate the underlying file system at that point.

Let's circle back to this point once we have a working FS? Because what this would do is (only) speed up multiple calls to open (and create, after #1858 ) when they occur on the same FileSystem. I.e. it speeds up the already-fast part of multi-part file read/write operations -- if it works.

Opened #1906.

return physicalFs.open(physicalPath, bufSize);
} catch (io.lakefs.clients.api.ApiException e) {
throw new RuntimeException("lakeFS API exception", e);
} catch (java.net.URISyntaxException e) {
throw new RuntimeException(e);
}
}

/**
Expand Down Expand Up @@ -189,54 +217,6 @@ public FileStatus[] globStatus(Path pathPattern) throws IOException {
return res;
}

/**
* An {@link InputStream} designated to serve as an input to the {@link FSDataInputStream} constructor. To be a
* viable input for FSDataInputStream, this class must be an instance of {@link InputStream} (StringBufferInputStream
* inherits it), and it must implement the interfaces {@link Seekable} and {@link PositionedReadable}.
*
* The read logic is implemented in {@link StringBufferInputStream#read()}.
*/
private class MyInputStream extends StringBufferInputStream implements Seekable,PositionedReadable {

public MyInputStream(String input) {
super(input);
LOG.debug("--------------------------- ctor ---------------------------");
}

@Override
public int read(long l, byte[] bytes, int i, int i1) throws IOException {
LOG.debug("--------------------------- read1 ---------------------------");
return 1;
}

@Override
public void readFully(long l, byte[] bytes, int i, int i1) throws IOException {
LOG.debug("--------------------------- readFully1---------------------------");
}

@Override
public void readFully(long l, byte[] bytes) throws IOException {
LOG.debug("--------------------------- readFully2 ---------------------------");
}

@Override
public void seek(long l) throws IOException {
LOG.debug("--------------------------- seek ---------------------------");
}

@Override
public long getPos() throws IOException {
LOG.debug("--------------------------- getPos---------------------------");
return 0;
}

@Override
public boolean seekToNewSource(long l) throws IOException {
LOG.debug("--------------------------- seekToNewSource---------------------------");
return false;
}
}

/**
* Returns Location with repository, ref and path used by lakeFS based on filesystem path.
* @param path
Expand Down