github.com/treeverse/lakefs@v1.24.1-0.20240520134607-95648127bfb0/clients/hadoopfs/src/main/java/io/lakefs/LakeFSFileSystem.java (about) 1 package io.lakefs; 2 3 import static io.lakefs.Constants.*; 4 5 import java.io.File; 6 import java.io.FileNotFoundException; 7 import java.io.IOException; 8 import java.io.OutputStream; 9 import java.net.URI; 10 import java.net.URISyntaxException; 11 import java.nio.file.AccessDeniedException; 12 import java.util.ArrayList; 13 import java.util.Collections; 14 import java.util.EnumSet; 15 import java.util.List; 16 import java.util.NoSuchElementException; 17 import java.util.concurrent.ExecutorService; 18 import java.util.concurrent.Executors; 19 import java.util.concurrent.ThreadFactory; 20 import java.util.concurrent.TimeUnit; 21 import java.util.stream.Collectors; 22 import javax.annotation.Nonnull; 23 import org.apache.commons.lang3.StringUtils; 24 import org.apache.hadoop.conf.Configuration; 25 import org.apache.hadoop.fs.BlockLocation; 26 import org.apache.hadoop.fs.CreateFlag; 27 import org.apache.hadoop.fs.FSDataInputStream; 28 import org.apache.hadoop.fs.FSDataOutputStream; 29 import org.apache.hadoop.fs.FileAlreadyExistsException; 30 import org.apache.hadoop.fs.FileStatus; 31 import org.apache.hadoop.fs.FileSystem; 32 import org.apache.hadoop.fs.LocatedFileStatus; 33 import org.apache.hadoop.fs.Path; 34 import org.apache.hadoop.fs.RemoteIterator; 35 import org.apache.hadoop.fs.permission.FsPermission; 36 import org.apache.hadoop.util.Progressable; 37 import org.apache.http.HttpStatus; 38 import org.slf4j.Logger; 39 import org.slf4j.LoggerFactory; 40 import io.lakefs.Constants.AccessMode; 41 import io.lakefs.clients.sdk.ApiException; 42 import io.lakefs.clients.sdk.BranchesApi; 43 import io.lakefs.clients.sdk.ObjectsApi; 44 import io.lakefs.clients.sdk.RepositoriesApi; 45 import io.lakefs.clients.sdk.model.ObjectCopyCreation; 46 import io.lakefs.clients.sdk.model.ObjectErrorList; 47 import io.lakefs.clients.sdk.model.ObjectStageCreation; 48 import io.lakefs.clients.sdk.model.ObjectStats; 49 import io.lakefs.clients.sdk.model.ObjectStatsList; 50 import io.lakefs.clients.sdk.model.Pagination; 51 import io.lakefs.clients.sdk.model.PathList; 52 import io.lakefs.clients.sdk.model.Repository; 53 import io.lakefs.clients.sdk.model.StorageConfig; 54 import io.lakefs.storage.CreateOutputStreamParams; 55 import io.lakefs.storage.PhysicalAddressTranslator; 56 import io.lakefs.storage.PresignedStorageAccessStrategy; 57 import io.lakefs.storage.SimpleStorageAccessStrategy; 58 import io.lakefs.storage.StorageAccessStrategy; 59 import io.lakefs.utils.ObjectLocation; 60 61 /** 62 * A dummy implementation of the core lakeFS Filesystem. 63 * This class implements a {@link LakeFSFileSystem} that can be registered to 64 * Spark and support limited write and read actions. 65 * <p> 66 * Configure Spark to use lakeFS filesystem by property: 67 * spark.hadoop.fs.lakefs.impl=io.lakefs.LakeFSFileSystem. 68 * <p> 69 * Configure the application or the filesystem application by properties: 70 * fs.lakefs.endpoint=http://localhost:8000/api/v1 71 * fs.lakefs.access.key=AKIAIOSFODNN7EXAMPLE 72 * fs.lakefs.secret.key=wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY 73 */ 74 public class LakeFSFileSystem extends FileSystem { 75 public static final Logger LOG = LoggerFactory.getLogger(LakeFSFileSystem.class); 76 public static final Logger OPERATIONS_LOG = LoggerFactory.getLogger(LakeFSFileSystem.class + "[OPERATION]"); 77 public static final String LAKEFS_DELETE_BULK_SIZE = "fs.lakefs.delete.bulk_size"; 78 79 private Configuration conf; 80 private URI uri; 81 private Path workingDirectory = new Path(Constants.SEPARATOR); 82 private ClientFactory clientFactory; 83 private LakeFSClient lfsClient; 84 private int listAmount; 85 private FileSystem fsForConfig; 86 private boolean failedFSForConfig = false; 87 private PhysicalAddressTranslator physicalAddressTranslator; 88 private StorageAccessStrategy storageAccessStrategy; 89 private AccessMode accessMode; 90 private static File emptyFile = new File("/dev/null"); 91 92 // Currently bulk deletes *must* receive a single-threaded executor! 93 private ExecutorService deleteExecutor = Executors.newSingleThreadExecutor(new ThreadFactory() { 94 @Override 95 public Thread newThread(Runnable r) { 96 Thread t = new Thread(r); 97 t.setDaemon(true); 98 return t; 99 } 100 }); 101 102 @Override 103 public URI getUri() { 104 return uri; 105 } 106 107 public interface ClientFactory { 108 LakeFSClient newClient() throws IOException; 109 } 110 111 @Override 112 public void initialize(URI name, Configuration conf) throws IOException { 113 initializeWithClientFactory(name, conf, new ClientFactory() { 114 public LakeFSClient newClient() throws IOException { return new LakeFSClient(name.getScheme(), conf); } 115 }); 116 } 117 118 void initializeWithClientFactory(URI name, Configuration conf, ClientFactory clientFactory) throws IOException { 119 super.initialize(name, conf); 120 this.uri = name; 121 this.conf = conf; 122 this.clientFactory = clientFactory; 123 this.lfsClient = clientFactory.newClient(); 124 125 String host = name.getHost(); 126 if (host == null) { 127 throw new IOException("Invalid repository specified"); 128 } 129 setConf(conf); 130 131 listAmount = FSConfiguration.getInt(conf, uri.getScheme(), LIST_AMOUNT_KEY_SUFFIX, DEFAULT_LIST_AMOUNT); 132 String accessModeConf = FSConfiguration.get(conf, uri.getScheme(), ACCESS_MODE_KEY_SUFFIX); 133 accessMode = AccessMode.valueOf(StringUtils.defaultIfBlank(accessModeConf, AccessMode.SIMPLE.toString()).toUpperCase()); 134 if (accessMode == AccessMode.PRESIGNED) { 135 storageAccessStrategy = new PresignedStorageAccessStrategy(this, lfsClient); 136 } else if (accessMode == AccessMode.SIMPLE) { 137 // setup address translator for simple storage access strategy 138 try { 139 StorageConfig storageConfig = lfsClient.getInternalApi().getStorageConfig().execute(); 140 physicalAddressTranslator = new PhysicalAddressTranslator(storageConfig.getBlockstoreType(), 141 storageConfig.getBlockstoreNamespaceValidityRegex()); 142 } catch (ApiException e) { 143 throw new IOException("Failed to get lakeFS blockstore type", e); 144 } 145 storageAccessStrategy = new SimpleStorageAccessStrategy(this, lfsClient, conf, physicalAddressTranslator); 146 } else { 147 throw new IOException("Invalid access mode: " + accessMode); 148 } 149 } 150 151 private synchronized FileSystem getFSForConfig() { 152 if (fsForConfig != null) { 153 return fsForConfig; 154 } 155 if (failedFSForConfig || accessMode == AccessMode.PRESIGNED || physicalAddressTranslator == null) { 156 return null; 157 } 158 Path path = new Path(uri); 159 ObjectLocation objectLoc = pathToObjectLocation(path); 160 RepositoriesApi repositoriesApi = lfsClient.getRepositoriesApi(); 161 try { 162 Repository repository = repositoriesApi.getRepository(objectLoc.getRepository()).execute(); 163 String storageNamespace = repository.getStorageNamespace(); 164 URI storageURI = URI.create(storageNamespace); 165 Path physicalPath = physicalAddressTranslator.translate(storageNamespace); 166 fsForConfig = physicalPath.getFileSystem(conf); 167 } catch (ApiException | URISyntaxException | IOException e) { 168 failedFSForConfig = true; 169 LOG.warn("get underlying filesystem for {}: {} (use default values)", path, e); 170 } 171 return fsForConfig; 172 } 173 174 @FunctionalInterface 175 private interface BiFunctionWithIOException<U, V, R> { 176 R apply(U u, V v) throws IOException; 177 } 178 179 @Override 180 public long getDefaultBlockSize(Path path) { 181 if (getFSForConfig() != null) { 182 return getFSForConfig().getDefaultBlockSize(path); 183 } 184 return Constants.DEFAULT_BLOCK_SIZE; 185 } 186 187 @Override 188 public long getDefaultBlockSize() { 189 if (getFSForConfig() != null) { 190 return getFSForConfig().getDefaultBlockSize(); 191 } 192 return Constants.DEFAULT_BLOCK_SIZE; 193 } 194 195 @Override 196 public FSDataInputStream open(Path path, int bufSize) throws IOException { 197 OPERATIONS_LOG.trace("open({})", path); 198 try { 199 ObjectLocation objectLoc = pathToObjectLocation(path); 200 return storageAccessStrategy.createDataInputStream(objectLoc, bufSize); 201 } catch (ApiException e) { 202 throw translateException("open: " + path, e); 203 } 204 } 205 206 @Override 207 public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException { 208 OPERATIONS_LOG.trace("list_files({}), recursive={}", f, recursive); 209 return new RemoteIterator<LocatedFileStatus>() { 210 private final ListingIterator iterator = new ListingIterator(f, recursive, listAmount); 211 212 @Override 213 public boolean hasNext() throws IOException { 214 return iterator.hasNext(); 215 } 216 217 @Override 218 public LocatedFileStatus next() throws IOException { 219 LakeFSFileStatus status = iterator.next(); 220 BlockLocation[] locations = status.isFile() 221 ? getFileBlockLocations(status, 0, status.getLen()) 222 : new BlockLocation[0]; 223 return new LocatedFileStatus(status, locations); 224 } 225 }; 226 } 227 228 /** 229 * {@inheritDoc} 230 * Called on a file write Spark/Hadoop action. This method writes the content of the file in path into stdout. 231 */ 232 @Override 233 public FSDataOutputStream create(Path path, FsPermission permission, boolean overwrite, 234 int bufferSize, short replication, long blockSize, 235 Progressable progress) throws IOException { 236 OPERATIONS_LOG.trace("create({})", path); 237 try { 238 LakeFSFileStatus status = getFileStatus(path); 239 if (status.isDirectory()) { 240 throw new FileAlreadyExistsException(path + " is a directory"); 241 } 242 if (!overwrite) { 243 throw new FileAlreadyExistsException(path + " already exists"); 244 } 245 } catch (FileNotFoundException ignored) { 246 } 247 try { 248 ObjectLocation objectLoc = pathToObjectLocation(path); 249 return storageAccessStrategy.createDataOutputStream(objectLoc, 250 new CreateOutputStreamParams() 251 .bufferSize(bufferSize) 252 .blockSize(blockSize) 253 .progress(progress) 254 ); 255 } catch (ApiException e) { 256 throw new IOException("staging.getPhysicalAddress: " + e.getResponseBody(), e); 257 } 258 } 259 260 @Override 261 public FSDataOutputStream append(Path path, int i, Progressable progressable) throws IOException { 262 throw new UnsupportedOperationException("Append is not supported by LakeFSFileSystem"); 263 } 264 265 /** 266 * Rename, behaving similarly to the POSIX "mv" command, but non-atomically. 267 * 1. Rename is only supported for uncommitted data on the same branch. 268 * 2. The following rename scenarios are supported: 269 * file -> existing-file-name: rename(src.txt, existing-dst.txt) -> existing-dst.txt, existing-dst.txt is overridden 270 * file -> existing-directory-name: rename(src.txt, existing-dstdir) -> existing-dstdir/src.txt 271 * file -> non-existing dst: in case of non-existing rename target, false is return. note that empty directory is 272 * considered an existing directory and rename will move the directory/file into that folder. 273 * directory -> existing directory: rename(srcDir(containing srcDir/a.txt), existing-dstdir) -> existing-dstdir/a.txt 274 * 3. Rename dst path can be an uncommitted file, that will be overridden as a result of the rename operation. 275 * 4. The 'mtime' of the src object is not preserved. 276 * 277 * @throws IOException 278 */ 279 @Override 280 public boolean rename(Path src, Path dst) throws IOException { 281 OPERATIONS_LOG.trace("rename {} to {}", src, dst); 282 ObjectLocation srcObjectLoc = pathToObjectLocation(src); 283 ObjectLocation dstObjectLoc = pathToObjectLocation(dst); 284 // Same as s3a https://github.com/apache/hadoop/blob/874c055e73293e0f707719ebca1819979fb211d8/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L460 285 if (srcObjectLoc.getPath().isEmpty()) { 286 LOG.error("rename: src {} is root directory", src); 287 return false; 288 } 289 if (dstObjectLoc.getPath().isEmpty()) { 290 LOG.error("rename: dst {} is root directory", dst); 291 return false; 292 } 293 294 if (srcObjectLoc.equals(dstObjectLoc)) { 295 LOG.debug("rename: src and dst refer to the same lakefs object location: {}", dst); 296 return true; 297 } 298 299 if (!srcObjectLoc.onSameBranch(dstObjectLoc)) { 300 LOG.error("rename: src {} and dst {} are not on the same branch. rename outside this scope is unsupported " 301 + "by lakefs.", src, dst); 302 return false; 303 } 304 305 // Return false when src does not exist. mimics s3a's behaviour in 306 // https://github.com/apache/hadoop/blob/874c055e73293e0f707719ebca1819979fb211d8/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L468 307 LakeFSFileStatus srcStatus; 308 try { 309 srcStatus = getFileStatus(src); 310 } catch (FileNotFoundException ignored) { 311 return false; 312 } 313 boolean result; 314 if (srcStatus.isDirectory()) { 315 result = renameDirectory(src, dst); 316 } else { 317 result = renameFile(srcStatus, dst); 318 } 319 if (!src.getParent().equals(dst.getParent())) { 320 deleteEmptyDirectoryMarkers(dst.getParent()); 321 createDirectoryMarkerIfNotExists(src.getParent()); 322 } 323 return result; 324 } 325 326 327 /** 328 * Recursively rename objects under src dir. 329 * 330 * @return true if all objects under src renamed successfully, false otherwise. 331 */ 332 private boolean renameDirectory(Path src, Path dst) throws IOException { 333 try { 334 // May be unnecessary with https://github.com/treeverse/lakeFS/issues/1691 335 LakeFSFileStatus dstFileStatus = getFileStatus(dst); 336 if (!dstFileStatus.isDirectory()) { 337 // Same as https://github.com/apache/hadoop/blob/874c055e73293e0f707719ebca1819979fb211d8/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L482 338 LOG.debug("renameDirectory: rename src {} to dst {}: src is a directory and dst is a file", src, dst); 339 return false; 340 } 341 // lakefsFs only has non-empty directories. Therefore, if the destination is an existing directory we consider 342 // it to be non-empty. The behaviour is same as https://github.com/apache/hadoop/blob/874c055e73293e0f707719ebca1819979fb211d8/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L482 343 if (!dstFileStatus.isEmptyDirectory()) { 344 LOG.debug("renameDirectory: rename src {} to dst {}: dst is a non-empty directory.", src, dst); 345 return false; 346 } 347 // delete empty directory marker from destination 348 // based on the same behaviour https://github.com/apache/hadoop/blob/874c055e73293e0f707719ebca1819979fb211d8/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L549 349 deleteHelper(pathToObjectLocation(dst).toDirectory()); 350 } catch (FileNotFoundException e) { 351 LOG.debug("renameDirectory: dst {} does not exist", dst); 352 // Ensure parent directory exists 353 if (!isDirectory(dst.getParent())) { 354 return false; 355 } 356 } 357 358 ListingIterator iterator = new ListingIterator(src, true, listAmount); 359 iterator.setRemoveDirectory(false); 360 while (iterator.hasNext()) { 361 // TODO (Tals): parallelize objects rename process. 362 LakeFSFileStatus fileStatus = iterator.next(); 363 Path objDst = buildObjPathOnNonExistingDestinationDir(fileStatus.getPath(), src, dst); 364 try { 365 renameObject(fileStatus, objDst); 366 } catch (IOException e) { 367 // Rename dir operation in non-transactional. if one object rename failed we will end up in an 368 // intermediate state. TODO: consider adding a cleanup similar to 369 // https://github.com/apache/hadoop/blob/2960d83c255a00a549f8809882cd3b73a6266b6d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java#L191 370 throw new IOException("renameDirectory: failed to rename src directory " + src, e); 371 } 372 } 373 return true; 374 } 375 376 /** 377 * Sample input and output 378 * input: 379 * renamedObj: lakefs://repo/main/dir1/file1.txt 380 * srcDirPath: lakefs://repo/main/dir1 381 * dstDirPath: lakefs://repo/main/dir2 382 * output: 383 * lakefs://repo/main/dir2/file1.txt 384 */ 385 private Path buildObjPathOnNonExistingDestinationDir(Path renamedObj, Path srcDir, Path dstDir) { 386 String renamedPath = renamedObj.toUri().getPath(); 387 String srcPath = srcDir.toUri().getPath(); 388 if (srcPath.length() == renamedPath.length()) { 389 // we rename a directory 390 return new Path(dstDir.toUri()); 391 } 392 String renamedObjName = renamedPath.substring(srcPath.length() + 1); 393 String newObjPath = dstDir.toUri() + SEPARATOR + renamedObjName; 394 return new Path(newObjPath); 395 } 396 397 /** 398 * Sample input and output 399 * input: 400 * srcObj: lakefs://repo/main/file1.txt 401 * dstDir: lakefs://repo/main/dir1 402 * output: 403 * lakefs://repo/main/dir1/file1.txt 404 * <p> 405 * input: 406 * srcObj: lakefs://repo/main/dir1/file1.txt 407 * dstDir: lakefs://repo/main/dir2/file2.txt 408 * output: 409 * lakefs://repo/main/dir2/file2.txt 410 */ 411 private Path buildObjPathOnExistingDestinationDir(Path srcObj, Path dstDir) { 412 Path srcParent = srcObj.getParent(); 413 String filename = srcObj.toString().substring(srcParent.toString().length() + SEPARATOR.length()); 414 return new Path(dstDir + SEPARATOR + filename); 415 } 416 417 private boolean renameFile(LakeFSFileStatus srcStatus, Path dst) throws IOException { 418 LakeFSFileStatus dstFileStatus; 419 try { 420 dstFileStatus = getFileStatus(dst); 421 LOG.debug("renameFile: dst {} exists and is a {}", dst, dstFileStatus.isDirectory() ? "directory" : "file"); 422 if (dstFileStatus.isDirectory()) { 423 dst = buildObjPathOnExistingDestinationDir(srcStatus.getPath(), dst); 424 LOG.debug("renameFile: use {} to create dst {}", srcStatus.getPath(), dst); 425 } 426 } catch (FileNotFoundException e) { 427 LOG.debug("renameFile: dst does not exist, renaming src {} to a file called dst {}", 428 srcStatus.getPath(), dst); 429 // Ensure parent directory exists 430 if (!isDirectory(dst.getParent())) { 431 return false; 432 } 433 } 434 return renameObject(srcStatus, dst); 435 } 436 437 /** 438 * Non-atomic rename operation. 439 * 440 * @return true if rename succeeded, false otherwise 441 */ 442 private boolean renameObject(LakeFSFileStatus srcStatus, Path dst) throws IOException { 443 ObjectLocation srcObjectLoc = pathToObjectLocation(srcStatus.getPath()); 444 ObjectLocation dstObjectLoc = pathToObjectLocation(dst); 445 if (srcStatus.isEmptyDirectory()) { 446 srcObjectLoc = srcObjectLoc.toDirectory(); 447 dstObjectLoc = dstObjectLoc.toDirectory(); 448 } 449 450 ObjectsApi objects = lfsClient.getObjectsApi(); 451 //TODO (Tals): Can we add metadata? we currently don't have an API to get the metadata of an object. 452 453 try { 454 ObjectCopyCreation creationReq = new ObjectCopyCreation() 455 .srcRef(srcObjectLoc.getRef()) 456 .srcPath(srcObjectLoc.getPath()); 457 objects.copyObject(dstObjectLoc.getRepository(), dstObjectLoc.getRef(), dstObjectLoc.getPath(), 458 creationReq).execute(); 459 } catch (ApiException e) { 460 throw translateException("renameObject: src:" + srcStatus.getPath() + ", dst: " + dst + 461 ", call to copyObject failed", e); 462 } 463 // delete src path 464 try { 465 objects.deleteObject(srcObjectLoc.getRepository(), srcObjectLoc.getRef(), srcObjectLoc.getPath()).execute(); 466 } catch (ApiException e) { 467 throw translateException("renameObject: src:" + srcStatus.getPath() + ", dst: " + dst + 468 ", failed to delete src", e); 469 } 470 return true; 471 } 472 473 /** 474 * Translate {@link ApiException} to an {@link IOException}. 475 * 476 * @param msg the message describing the exception 477 * @param e the exception to translate 478 * @return an IOException that corresponds to the translated API exception 479 */ 480 private IOException translateException(String msg, ApiException e) { 481 int code = e.getCode(); 482 switch (code) { 483 case HttpStatus.SC_NOT_FOUND: 484 return (FileNotFoundException) new FileNotFoundException(msg).initCause(e); 485 case HttpStatus.SC_FORBIDDEN: 486 return (AccessDeniedException) new AccessDeniedException(msg).initCause(e); 487 default: 488 return new IOException(msg, e); 489 } 490 } 491 492 @Override 493 public boolean delete(Path path, boolean recursive) throws IOException { 494 OPERATIONS_LOG.trace("delete({}), recursive={}", path, recursive); 495 LakeFSFileStatus status; 496 try { 497 status = getFileStatus(path); 498 } catch (FileNotFoundException ignored) { 499 return false; 500 } 501 502 boolean deleted = true; 503 ObjectLocation loc = pathToObjectLocation(path); 504 if (status.isDirectory()) { 505 if (!recursive && !status.isEmptyDirectory()) { 506 throw new IOException("Path is a non-empty directory: " + path); 507 } 508 509 if (status.isEmptyDirectory()) { 510 loc = loc.toDirectory(); 511 deleted = deleteHelper(loc); 512 } else { 513 ObjectLocation location = pathToObjectLocation(path); 514 try (BulkDeleter deleter = newDeleter(location.getRepository(), location.getRef())) { 515 ListingIterator iterator = new ListingIterator(path, true, listAmount); 516 iterator.setRemoveDirectory(false); 517 while (iterator.hasNext()) { 518 LakeFSFileStatus fileStatus = iterator.next(); 519 ObjectLocation fileLoc = pathToObjectLocation(fileStatus.getPath()); 520 if (fileStatus.isDirectory()) { 521 fileLoc = fileLoc.toDirectory(); 522 } 523 deleter.add(fileLoc.getPath()); 524 } 525 } catch (BulkDeleter.DeleteFailuresException e) { 526 LOG.error("delete(%s, %b): %s", path, recursive, e.toString()); 527 deleted = false; 528 } 529 } 530 } else { 531 deleted = deleteHelper(loc); 532 } 533 534 createDirectoryMarkerIfNotExists(path.getParent()); 535 return deleted; 536 } 537 538 private BulkDeleter newDeleter(String repository, String branch) throws IOException { 539 // Use a different client -- a different thread waits for its calls, 540 // *late*. 541 ObjectsApi objectsApi = clientFactory.newClient().getObjectsApi(); 542 return new BulkDeleter(deleteExecutor, new BulkDeleter.Callback() { 543 public ObjectErrorList apply(String repository, String branch, PathList pathList) throws ApiException { 544 return objectsApi.deleteObjects(repository, branch, pathList).execute(); 545 } 546 }, repository, branch, conf.getInt(LAKEFS_DELETE_BULK_SIZE, 0)); 547 } 548 549 private boolean deleteHelper(ObjectLocation loc) throws IOException { 550 try { 551 ObjectsApi objectsApi = lfsClient.getObjectsApi(); 552 objectsApi.deleteObject(loc.getRepository(), loc.getRef(), loc.getPath()).execute(); 553 } catch (ApiException e) { 554 // This condition mimics s3a behaviour in 555 // https://github.com/apache/hadoop/blob/874c055e73293e0f707719ebca1819979fb211d8/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L619 556 if (e.getCode() == HttpStatus.SC_NOT_FOUND) { 557 LOG.error("Could not delete: {}, reason: {}", loc, e.getResponseBody()); 558 return false; 559 } 560 throw new IOException("deleteObject", e); 561 } 562 return true; 563 } 564 565 /** 566 * Delete parent directory markers from path until root. 567 * Assumption: the caller has created an object under the path, so the empty 568 * directory markers are no longer necessary. 569 * Based on the S3AFileSystem implementation. 570 * Note: there is a race here if this is called on a path which mkdir is trying to create. 571 * 572 * @param f path to start for empty directory markers 573 */ 574 void deleteEmptyDirectoryMarkers(Path f) { 575 while (true) { 576 try { 577 ObjectLocation objectLocation = pathToObjectLocation(f); 578 if (!objectLocation.isValidPath()) { 579 break; 580 } 581 582 LakeFSFileStatus status = getFileStatus(f); 583 if (status.isDirectory() && status.isEmptyDirectory()) { 584 deleteHelper(objectLocation.toDirectory()); 585 } else { 586 // not an empty directory, so the parent cannot be empty either 587 break; 588 } 589 } catch (IOException ignored) { 590 } 591 592 if (f.isRoot()) { 593 break; 594 } 595 596 f = f.getParent(); 597 } 598 } 599 600 /** 601 * create marker object for empty directory 602 * @param f path to check if empty directory marker is needed 603 * @throws IOException any issue with lakeFS or underlying filesystem 604 */ 605 private void createDirectoryMarkerIfNotExists(Path f) throws IOException { 606 ObjectLocation objectLocation = pathToObjectLocation(f).toDirectory(); 607 if (!objectLocation.isValidPath()) { 608 LOG.warn("Cannot create directory marker for invalid path {}", f.toString()); 609 // Safe to do nothing, because directory markers are mostly 610 // useless. This happens when the path inside the branch is 611 // empty -- and cannot be created. If the repo or branch names 612 // are empty this also happens but then the actual operation 613 // will fail. 614 return; 615 } 616 try { 617 ObjectsApi objects = lfsClient.getObjectsApi(); 618 ObjectStatsList osl = objects.listObjects(objectLocation.getRepository(), objectLocation.getRef()) 619 .userMetadata(false).presign(false).amount(5).prefix(objectLocation.getPath()) 620 .execute(); 621 List<ObjectStats> l = osl.getResults(); 622 if (l.isEmpty()) { 623 // No object with any name that starts with objectLocation - create a directory marker in place. 624 objects.uploadObject(objectLocation.getRepository(), objectLocation.getRef(), objectLocation.getPath()) 625 .ifNoneMatch("*").content(emptyFile) 626 .execute(); 627 } 628 629 } catch (ApiException e) { 630 if (e.getCode() == HttpStatus.SC_PRECONDITION_FAILED) { 631 LOG.trace("createDirectoryMarkerIfNotExists: Ignore {} response, marker exists"); 632 return; 633 } 634 throw new IOException("createDirectoryMarkerIfNotExists", e); 635 } 636 } 637 638 @Override 639 public FileStatus[] listStatus(Path path) throws FileNotFoundException, IOException { 640 OPERATIONS_LOG.trace("list_status({})", path); 641 ObjectLocation objectLoc = pathToObjectLocation(path); 642 ObjectsApi objectsApi = lfsClient.getObjectsApi(); 643 try { 644 ObjectStats objectStat = objectsApi.statObject(objectLoc.getRepository(), objectLoc.getRef(), objectLoc.getPath()).userMetadata(false).presign(false).execute(); 645 LakeFSFileStatus fileStatus = convertObjectStatsToFileStatus(objectLoc, objectStat); 646 return new FileStatus[]{fileStatus}; 647 } catch (ApiException e) { 648 if (e.getCode() != HttpStatus.SC_NOT_FOUND) { 649 throw new IOException("statObject", e); 650 } 651 } 652 // list directory content 653 List<FileStatus> fileStatuses = new ArrayList<>(); 654 ListingIterator iterator = new ListingIterator(path, false, listAmount); 655 while (iterator.hasNext()) { 656 LakeFSFileStatus fileStatus = iterator.next(); 657 fileStatuses.add(fileStatus); 658 } 659 if (!fileStatuses.isEmpty()) { 660 return fileStatuses.toArray(new FileStatus[0]); 661 } 662 // nothing to list - check if it is an empty directory marker 663 try { 664 ObjectStats objectStat = objectsApi.statObject(objectLoc.getRepository(), 665 objectLoc.getRef(), 666 objectLoc.getPath() + SEPARATOR) 667 .userMetadata(false).presign(false) 668 .execute(); 669 LakeFSFileStatus fileStatus = convertObjectStatsToFileStatus(objectLoc, objectStat); 670 if (fileStatus.isEmptyDirectory()) { 671 return new FileStatus[0]; 672 } 673 } catch (ApiException e) { 674 if (e.getCode() != HttpStatus.SC_NOT_FOUND) { 675 throw new IOException("statObject", e); 676 } 677 } 678 throw new FileNotFoundException("No such file or directory: " + path); 679 } 680 681 @Override 682 public void setWorkingDirectory(Path path) { 683 this.workingDirectory = path; 684 } 685 686 @Override 687 public Path getWorkingDirectory() { 688 return this.workingDirectory; 689 } 690 691 /** 692 * Make the given path and all non-existent parents into directories. 693 * We use the same technic as S3A implementation, an object size 0, without a name with delimiter ('/') that 694 * keeps the directory exists. 695 * When we write an object into the directory - we can delete the marker. 696 * @param path path to create 697 * @param fsPermission to apply (passing to the underlying filesystem) 698 * @return an IOException that corresponds to the translated API exception 699 */ 700 @Override 701 public boolean mkdirs(Path path, FsPermission fsPermission) throws IOException { 702 OPERATIONS_LOG.trace("mkdirs({})", path); 703 try { 704 // Check that path is not already a directory 705 FileStatus fileStatus = getFileStatus(path); 706 if (fileStatus.isDirectory()) { 707 return true; 708 } 709 throw new FileAlreadyExistsException("Path is a file: " + path); 710 } catch (FileNotFoundException e) { 711 // check if part of path is a file already 712 ObjectLocation objectLocation = pathToObjectLocation(path); 713 Path branchRoot = new Path(objectLocation.toRefString()); 714 Path currentPath = path; 715 do { 716 try { 717 FileStatus fileStatus = getFileStatus(currentPath); 718 if (fileStatus.isFile()) { 719 throw new FileAlreadyExistsException(String.format( 720 "Can't make directory for path '%s' since it is a file.", 721 currentPath)); 722 } 723 } catch (FileNotFoundException ignored) { 724 } 725 currentPath = currentPath.getParent(); 726 } while (currentPath != null && !currentPath.equals(branchRoot)); 727 728 createDirectoryMarker(path); 729 return true; 730 } 731 } 732 733 private void createDirectoryMarker(Path path) throws IOException { 734 try { 735 ObjectLocation objectLoc = pathToObjectLocation(path).toDirectory(); 736 OutputStream out = storageAccessStrategy.createDataOutputStream(objectLoc, null); 737 out.close(); 738 } catch (io.lakefs.clients.sdk.ApiException e) { 739 throw new IOException("createDirectoryMarker: " + e.getResponseBody(), e); 740 } 741 } 742 743 /** 744 * Return a file status object that represents the path. 745 * @param path to a file or directory 746 * @return a LakeFSFileStatus object 747 * @throws java.io.FileNotFoundException when the path does not exist; 748 * IOException API call or underlying filesystem exceptions 749 */ 750 @Override 751 public LakeFSFileStatus getFileStatus(Path path) throws IOException { 752 OPERATIONS_LOG.trace("get_file_status({})", path); 753 ObjectLocation objectLoc = pathToObjectLocation(path); 754 if (objectLoc.getPath().isEmpty()) { 755 if (isBranchExists(objectLoc.getRepository(), objectLoc.getRef())) { 756 return new LakeFSFileStatus.Builder(path).isdir(true).build(); 757 } 758 throw new FileNotFoundException(path + " not found"); 759 } 760 ObjectsApi objectsApi = lfsClient.getObjectsApi(); 761 // get object status on path 762 try { 763 ObjectStats os = objectsApi.statObject(objectLoc.getRepository(), 764 objectLoc.getRef(), 765 objectLoc.getPath()) 766 .userMetadata(false) 767 .presign(false) 768 .execute(); 769 return convertObjectStatsToFileStatus(objectLoc, os); 770 } catch (ApiException e) { 771 if (e.getCode() != HttpStatus.SC_NOT_FOUND) { 772 throw new IOException("statObject", e); 773 } 774 } 775 // get object status on path + "/" for directory marker directory 776 try { 777 ObjectStats objectStat = objectsApi.statObject(objectLoc.getRepository(), 778 objectLoc.getRef(), 779 objectLoc.getPath() + SEPARATOR) 780 .userMetadata(false).presign(false) 781 .execute(); 782 return convertObjectStatsToFileStatus(objectLoc, objectStat); 783 } catch (ApiException e) { 784 if (e.getCode() != HttpStatus.SC_NOT_FOUND) { 785 throw new IOException("statObject", e); 786 } 787 } 788 // not found as a file or directory marker; check if path is a "directory", i.e. a prefix. 789 ListingIterator iterator = new ListingIterator(path, true, 1); 790 iterator.setRemoveDirectory(false); 791 if (iterator.hasNext()) { 792 Path filePath = new Path(objectLoc.toString()); 793 return new LakeFSFileStatus.Builder(filePath).isdir(true).build(); 794 } 795 throw new FileNotFoundException(path + " not found"); 796 } 797 798 @Nonnull 799 private LakeFSFileStatus convertObjectStatsToFileStatus(ObjectLocation objectLocation, ObjectStats objectStat) throws IOException { 800 long length = 0; 801 Long sizeBytes = objectStat.getSizeBytes(); 802 if (sizeBytes != null) { 803 length = sizeBytes; 804 } 805 long modificationTime = 0; 806 Long mtime = objectStat.getMtime(); 807 if (mtime != null) { 808 modificationTime = TimeUnit.SECONDS.toMillis(mtime); 809 } 810 Path filePath = new Path(ObjectLocation.formatPath(objectLocation.getScheme(), objectLocation.getRepository(), 811 objectLocation.getRef(), objectStat.getPath())); 812 String physicalAddress = objectStat.getPhysicalAddress(); 813 boolean isDir = isDirectory(objectStat); 814 boolean isEmptyDirectory = isDir && objectStat.getPathType() == ObjectStats.PathTypeEnum.OBJECT; 815 long blockSize = isDir 816 ? 0 817 : getDefaultBlockSize(); 818 LakeFSFileStatus.Builder builder = 819 new LakeFSFileStatus.Builder(filePath) 820 .length(length) 821 .isdir(isDir) 822 .isEmptyDirectory(isEmptyDirectory) 823 .blockSize(blockSize) 824 .mTime(modificationTime) 825 .checksum(objectStat.getChecksum()) 826 .physicalAddress(physicalAddress); 827 return builder.build(); 828 } 829 830 /** 831 * Return the protocol scheme for the FileSystem. 832 * 833 * @return lakefs scheme 834 */ 835 @Override 836 public String getScheme() { 837 return this.uri.getScheme(); 838 } 839 840 @Override 841 public boolean exists(Path path) throws IOException { 842 OPERATIONS_LOG.trace("exists({})", path); 843 ObjectLocation objectLoc = pathToObjectLocation(path); 844 // no path - check if branch exists 845 if (objectLoc.getPath().isEmpty()) { 846 return isBranchExists(objectLoc.getRepository(), objectLoc.getRef()); 847 } 848 849 ObjectsApi objects = lfsClient.getObjectsApi(); 850 /* 851 * path "exists" in Hadoop if one of these holds: 852 * 853 * - path exists on lakeFS (regular file) 854 * - path + "/" exists (directory marker) 855 * - path + "/" + <something> exists (a nonempty directory that has no 856 * directory marker for some reason; perhaps it was not created by 857 * Spark). 858 */ 859 860 String directoryPath = objectLoc.toDirectory().getPath(); 861 // List a small number of objects after path. If either path or 862 // path + "/" + <something> are there, then path exists. Pick the 863 // number of objects so that it costs about the same to list that 864 // many objects as it does to list 1. 865 try { 866 // TODO(ariels,itaiad200): configure the "5" to the right value. 867 // "Right" being: the number of objects that costs about the same to list as 1. 868 // 5 is a good guess for now since that in DynamoDB backends listing 5 objects cost the same as 1. 869 ObjectStatsList osl = objects.listObjects(objectLoc.getRepository(), objectLoc.getRef()) 870 .userMetadata(false).presign(false).amount(5).prefix(objectLoc.getPath()) 871 .execute(); 872 List<ObjectStats> l = osl.getResults(); 873 if (l.isEmpty()) { 874 // No object with any name that starts with objectLoc. 875 return false; 876 } 877 ObjectStats first = l.get(0); 878 if (first.getPath().equals(objectLoc.getPath())) { 879 // objectLoc itself points at the object, it's a regular object! 880 return true; 881 } 882 for (ObjectStats stat : l) { 883 if (stat.getPath().startsWith(directoryPath)) { 884 // path exists as a directory containing this object. 885 // Also handles the case where this object is a directory marker. 886 return true; 887 } 888 if (stat.getPath().compareTo(directoryPath) > 0) { 889 // This object is after path + "/" and does not start 890 // with it: there can be no object under path + "/". 891 return false; 892 } 893 } 894 if (!osl.getPagination().getHasMore()) { 895 // Scanned all files with prefix path and did not find 896 // anything with path or path + "/". 897 return false; 898 } 899 } catch (ApiException e) { 900 if (e.getCode() == HttpStatus.SC_NOT_FOUND) { 901 // Repository or ref do not exist. 902 return false; 903 } else { 904 throw new IOException("exists", e); 905 } 906 } 907 908 // The initial listing did not even reach path+"/". We know path 909 // does not exist (it would have been first in that listing), so 910 // just check if path+"/" or something below it exist. 911 912 try { 913 ObjectStatsList osl = objects.listObjects(objectLoc.getRepository(), objectLoc.getRef()) 914 .userMetadata(false).presign(false).amount(1).prefix(directoryPath) 915 .execute(); 916 List<ObjectStats> l = osl.getResults(); 917 return ! l.isEmpty(); 918 } catch (ApiException e) { 919 // Repo and ref exist! 920 throw new IOException("exists", e); 921 } 922 } 923 924 private boolean isBranchExists(String repository, String branch) throws IOException { 925 try { 926 BranchesApi branches = lfsClient.getBranchesApi(); 927 branches.getBranch(repository, branch).execute(); 928 return true; 929 } catch (ApiException e) { 930 if (e.getCode() != HttpStatus.SC_NOT_FOUND) { 931 throw new IOException("getBranch", e); 932 } 933 return false; 934 } 935 } 936 937 /** 938 * Returns Location with repository, ref and path used by lakeFS based on filesystem path. 939 * 940 * @param path to extract information from. 941 * @return lakeFS Location with repository, ref and path 942 */ 943 @Nonnull 944 public ObjectLocation pathToObjectLocation(Path path) { 945 return ObjectLocation.pathToObjectLocation(this.workingDirectory, path); 946 } 947 948 class ListingIterator implements RemoteIterator<LakeFSFileStatus> { 949 private final ObjectLocation objectLocation; 950 private final String delimiter; 951 private final int amount; 952 private boolean removeDirectory; 953 private String nextOffset; 954 private boolean last; 955 private List<ObjectStats> chunk; 956 private int pos; 957 958 /** 959 * Returns iterator for files under path. 960 * When recursive is set, the iterator will list all files under the target path (delimiter is ignored). 961 * Parameter amount controls the limit for each request for listing. 962 * 963 * @param path the location to list 964 * @param recursive boolean for recursive listing 965 * @param amount buffer size to fetch listing 966 */ 967 public ListingIterator(Path path, boolean recursive, int amount) { 968 this.removeDirectory = recursive; 969 this.chunk = Collections.emptyList(); 970 // we assume that 'path' is a directory 971 this.objectLocation = pathToObjectLocation(path).toDirectory(); 972 this.delimiter = recursive ? "" : SEPARATOR; 973 this.last = false; 974 this.pos = 0; 975 this.amount = amount == 0 ? DEFAULT_LIST_AMOUNT : amount; 976 this.nextOffset = ""; 977 } 978 979 @Override 980 public boolean hasNext() throws IOException { 981 // read next chunk if needed 982 if (!this.last && this.pos >= this.chunk.size()) { 983 this.readNextChunk(); 984 } 985 // return if there is next item available 986 return this.pos < this.chunk.size(); 987 } 988 989 private void readNextChunk() throws IOException { 990 String listingPath = this.objectLocation.getPath(); 991 do { 992 try { 993 ObjectsApi objectsApi = lfsClient.getObjectsApi(); 994 ObjectStatsList resp = objectsApi.listObjects(objectLocation.getRepository(), objectLocation.getRef()) 995 .userMetadata(false).presign(false).after(nextOffset).amount(amount).delimiter(delimiter).prefix(objectLocation.getPath()) 996 .execute(); 997 chunk = resp.getResults(); 998 pos = 0; 999 Pagination pagination = resp.getPagination(); 1000 nextOffset = pagination.getNextOffset(); 1001 if (!pagination.getHasMore()) { 1002 last = true; 1003 } 1004 } catch (ApiException e) { 1005 throw new IOException("listObjects", e); 1006 } 1007 chunk = chunk.stream().filter(item -> { 1008 // filter directories if needed 1009 if (this.removeDirectory && isDirectory(item)) { 1010 return false; 1011 } 1012 // filter out the marker object of the path we list 1013 return !item.getPath().equals(listingPath); 1014 }).collect(Collectors.toList()); 1015 // loop until we have something or last chunk 1016 } while (chunk.isEmpty() && !last); 1017 } 1018 1019 public boolean isRemoveDirectory() { 1020 return removeDirectory; 1021 } 1022 1023 public void setRemoveDirectory(boolean removeDirectory) { 1024 this.removeDirectory = removeDirectory; 1025 } 1026 1027 @Override 1028 public LakeFSFileStatus next() throws IOException { 1029 if (!hasNext()) { 1030 throw new NoSuchElementException("No more entries"); 1031 } 1032 ObjectStats objectStats = chunk.get(pos++); 1033 return convertObjectStatsToFileStatus( 1034 objectLocation, 1035 objectStats); 1036 } 1037 } 1038 1039 private static boolean isDirectory(ObjectStats stat) { 1040 return stat.getPath().endsWith(SEPARATOR) || stat.getPathType() == ObjectStats.PathTypeEnum.COMMON_PREFIX; 1041 } 1042 public FSDataOutputStream createNonRecursive(Path path, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { 1043 Path parent = path.getParent(); 1044 if (parent != null && !this.getFileStatus(parent).isDirectory()) { 1045 throw new FileAlreadyExistsException("Not a directory: " + parent); 1046 } else { 1047 return this.create(path, permission, flags.contains(CreateFlag.OVERWRITE), bufferSize, replication, blockSize, progress); 1048 } 1049 } 1050 }