github.com/treeverse/lakefs@v1.24.1-0.20240520134607-95648127bfb0/clients/hadoopfs/src/main/java/io/lakefs/storage/SimpleStorageAccessStrategy.java (about) 1 package io.lakefs.storage; 2 3 import java.io.IOException; 4 import java.io.OutputStream; 5 import java.net.URISyntaxException; 6 import java.util.Objects; 7 8 import org.apache.hadoop.conf.Configuration; 9 import org.apache.hadoop.fs.FSDataInputStream; 10 import org.apache.hadoop.fs.FSDataOutputStream; 11 import org.apache.hadoop.fs.FileSystem; 12 import org.apache.hadoop.fs.Path; 13 14 import io.lakefs.LakeFSClient; 15 import io.lakefs.LakeFSFileSystem; 16 import io.lakefs.LakeFSLinker; 17 import io.lakefs.clients.sdk.ApiException; 18 import io.lakefs.clients.sdk.ObjectsApi; 19 import io.lakefs.clients.sdk.StagingApi; 20 import io.lakefs.clients.sdk.model.ObjectStats; 21 import io.lakefs.clients.sdk.model.StagingLocation; 22 import io.lakefs.utils.ObjectLocation; 23 24 public class SimpleStorageAccessStrategy implements StorageAccessStrategy { 25 private final PhysicalAddressTranslator physicalAddressTranslator; 26 private final LakeFSFileSystem lakeFSFileSystem; 27 private final LakeFSClient lfsClient; 28 private final Configuration conf; 29 30 public SimpleStorageAccessStrategy(LakeFSFileSystem lakeFSFileSystem, LakeFSClient lfsClient, 31 Configuration conf, PhysicalAddressTranslator physicalAddressTranslator) { 32 this.lakeFSFileSystem = lakeFSFileSystem; 33 this.lfsClient = lfsClient; 34 this.conf = conf; 35 this.physicalAddressTranslator = physicalAddressTranslator; 36 } 37 38 @Override 39 public FSDataOutputStream createDataOutputStream(ObjectLocation objectLocation, 40 CreateOutputStreamParams params, boolean overwrite) throws ApiException, IOException { 41 StagingApi staging = lfsClient.getStagingApi(); 42 StagingLocation stagingLocation = 43 staging.getPhysicalAddress(objectLocation.getRepository(), 44 objectLocation.getRef(), objectLocation.getPath()) 45 .presign(false) 46 .execute(); 47 Path physicalPath; 48 try { 49 physicalPath = physicalAddressTranslator.translate(Objects.requireNonNull(stagingLocation.getPhysicalAddress())); 50 } catch (URISyntaxException e) { 51 throw new IOException("Failed to parse object phystical address", e); 52 } 53 FileSystem physicalFs = physicalPath.getFileSystem(conf); 54 OutputStream physicalOut; 55 if (params != null) { 56 physicalOut = physicalFs.create(physicalPath, false, params.bufferSize, 57 physicalFs.getDefaultReplication(physicalPath), params.blockSize); 58 } else { 59 physicalOut = physicalFs.create(physicalPath); 60 } 61 MetadataClient metadataClient = new MetadataClient(physicalFs); 62 LakeFSLinker linker = new LakeFSLinker(lakeFSFileSystem, lfsClient, objectLocation, stagingLocation, overwrite); 63 LinkOnCloseOutputStream out = new LinkOnCloseOutputStream(physicalPath.toUri(), metadataClient, physicalOut, linker); 64 // TODO(ariels): add fs.FileSystem.Statistics here to keep track. 65 return new FSDataOutputStream(out, null); 66 } 67 68 @Override 69 public FSDataInputStream createDataInputStream(ObjectLocation objectLocation, int bufSize) 70 throws ApiException, IOException { 71 ObjectsApi objects = lfsClient.getObjectsApi(); 72 ObjectStats stats = objects.statObject(objectLocation.getRepository(), 73 objectLocation.getRef(), objectLocation.getPath()) 74 .userMetadata(false).presign(false) 75 .execute(); 76 Path physicalPath; 77 try { 78 physicalPath = physicalAddressTranslator.translate(Objects.requireNonNull(stats.getPhysicalAddress())); 79 } catch (URISyntaxException e) { 80 throw new IOException("Failed to parse object physical address", e); 81 } 82 FileSystem physicalFs = physicalPath.getFileSystem(conf); 83 return physicalFs.open(physicalPath, bufSize); 84 } 85 }