github.com/treeverse/lakefs@v1.24.1-0.20240520134607-95648127bfb0/clients/hadoopfs/src/main/java/io/lakefs/FileSystemTracer.java (about)

     1  package io.lakefs;
     2  
     3  import io.lakefs.utils.ObjectLocation;
     4  import io.lakefs.utils.StringUtils;
     5  
     6  import org.apache.hadoop.conf.Configuration;
     7  import org.apache.hadoop.fs.*;
     8  import org.apache.hadoop.fs.permission.FsPermission;
     9  import org.apache.hadoop.fs.s3a.S3AFileSystem;
    10  import org.apache.hadoop.util.Progressable;
    11  import org.jetbrains.annotations.NotNull;
    12  import org.slf4j.Logger;
    13  import org.slf4j.LoggerFactory;
    14  
    15  import java.io.FileNotFoundException;
    16  import java.io.IOException;
    17  import java.io.OutputStream;
    18  import java.net.URI;
    19  
    20  /**
    21   * This class implements a troubleshooting tool for the LakeFSFileSystem. You can use it to compare the output of file
    22   * system operations done by {@link LakeFSFileSystem} and by {@link S3AFileSystem}.
    23   *
    24   * How to use the FileSystemTracer:
    25   * To use the FileSystemTracer, you should set the value of the Hadoop configuration 'fs.lakefs.impl' to FileSystemTracer.
    26   * This configures the FileSystemTracer to be the file system that handles paths with the lakefs scheme. i.e. paths with
    27   * lakefs:// prefix.
    28   *
    29   * How does the FileSystemTracer work:
    30   * The FileSystemTracer holds instances of {@link LakeFSFileSystem} and {@link S3AFileSystem}. On a file system operation,
    31   * the FileSystemTracer invokes the operation on both file systems, logs the output of both calls, and returns the result
    32   * of one of the file systems based on a configuration (S3AFileSystem output by default).
    33  
    34   * Configuration:
    35   * - fs.lakefs.tracer.working_dir - the s3 location in which the tracer can operate. This should be an S3 bucket name or
    36   *   an absolute path of a directory on a bucket.
    37   * Optional -
    38   * - fs.lakefs.tracer.use_lakefs_output - tells the tracer whether it should return the response coming from the lakefs file
    39   *   system or return s3a's response. by default it is set to false and returns s3a's response.
    40   *
    41   * Assumptions:
    42   * - The content of lakefs://repository/branch/ and s3a://${fs.lakefs.tracer.working.dir}/ should be identical.
    43   * - The s3 credentials available for Spark allow access to fs.lakefs.tracer.working.dir on s3.
    44   */
    45  public class FileSystemTracer extends FileSystem {
    46  
    47      private static final Logger LOG = LoggerFactory.getLogger(FileSystemTracer.class);
    48      private static final String TRACER_WORKING_DIR = "fs.lakefs.tracer.working_dir";
    49      private static final String USE_LAKEFS_OUTPUT = "fs.lakefs.tracer.use_lakefs_output";
    50      private static final Object S3_URI_SCHEME = "s3";
    51      private static final Object RESULTS_COMPARISON = "[RESULTS_COMPARISON]";
    52  
    53  
    54      /*
    55       A property that determines which file system's response the FileSystemTracer returns.
    56       */
    57      private boolean useLakeFSFileSystemResults;
    58      private LakeFSFileSystem lfsFileSystem;
    59      private FileSystem s3AFileSystem;
    60      private String s3aPathPrefix;
    61      private String lfsPathPrefix;
    62  
    63      /**
    64       * Transforms a lakefs path into an s3 path.
    65       * Example:
    66       *   in: lakefs://repository/branch/key=1/a.parquet
    67       *   out: s3://${fs.lakefs.tracer.working_dir}/key=1/a.parquet
    68       */
    69      private Path translateLakeFSPathToS3APath(Path path) {
    70          return replacePathPrefix(path, lfsPathPrefix, s3aPathPrefix);
    71      }
    72  
    73      /**
    74       * Transforms an S3A path into a lakefs path.
    75       * This method is used to transform the output of S3AFileSystem operations that include paths, so that Spark
    76       * continues to direct it's requests to the FileSystemTracer that handles paths with the "lakefs://" prefix.
    77       * Example:
    78       *   in: s3://${fs.lakefs.tracer.working_dir}/key=1/a.parquet
    79       *   out: lakefs://repository/branch/key=1/a.parquet
    80       */
    81      private Path translateS3APathToLakeFSPath(Path path) {
    82          return replacePathPrefix(path, s3aPathPrefix, lfsPathPrefix);
    83      }
    84  
    85      private Path replacePathPrefix(Path path, String curPrefix, String newPrefix) {
    86          String p = path.toString();
    87          boolean isValidPath = p.startsWith(curPrefix);
    88          if (isValidPath) {
    89              String objRelativePath = StringUtils.trimLeadingSlash(p.substring(curPrefix.length()));
    90              String newPath = String.format("%s/%s", newPrefix, objRelativePath);
    91              LOG.trace("Converted {} to {}", path, newPath);
    92              return new Path(newPath);
    93          }
    94          LOG.error("Invalid path {}", path);
    95          return null;
    96      }
    97  
    98      @Override
    99      public void initialize(URI name, Configuration conf) throws IOException {
   100          lfsFileSystem = new LakeFSFileSystem();
   101          lfsFileSystem.initialize(name, conf);
   102          ObjectLocation loc = lfsFileSystem.pathToObjectLocation(new Path(name));
   103          lfsPathPrefix = ObjectLocation.formatPath(loc.getScheme(), loc.getRepository(), loc.getRef());
   104  
   105          String tracerWorkingDir = conf.get(TRACER_WORKING_DIR);
   106          if (tracerWorkingDir == null) {
   107              throw new IOException("tracerWorkingDir is null");
   108          }
   109          s3aPathPrefix = String.format("%s://%s", S3_URI_SCHEME, tracerWorkingDir);
   110          Path s3aPath = new Path(s3aPathPrefix);
   111          s3AFileSystem = s3aPath.getFileSystem(conf);
   112  
   113          useLakeFSFileSystemResults = conf.getBoolean(USE_LAKEFS_OUTPUT, false);
   114          LOG.trace("Initialization finished, fs.lakefs.tracer.use_lakefs_output: {}", useLakeFSFileSystemResults);
   115      }
   116  
   117      @Override
   118      public URI getUri() {
   119          LOG.trace("getUri");
   120  
   121          URI lakefsRes = lfsFileSystem.getUri();
   122          URI s3aRes = s3AFileSystem.getUri();
   123          LOG.trace("{}[getUri] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   124  
   125          if (useLakeFSFileSystemResults) {
   126              return lakefsRes;
   127          }
   128          return s3aRes;
   129      }
   130  
   131      @Override
   132      public Path makeQualified(Path path) {
   133          LOG.trace("makeQualified");
   134  
   135          Path lakefsRes = lfsFileSystem.makeQualified(path);
   136          Path s3aRes = s3AFileSystem.makeQualified(translateLakeFSPathToS3APath(path));
   137          LOG.trace("{}[makeQualified] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   138  
   139          if (useLakeFSFileSystemResults) {
   140              return lakefsRes;
   141          }
   142          return translateS3APathToLakeFSPath(s3aRes);
   143      }
   144  
   145      @Override
   146      public FSDataInputStream open(Path f, int bufferSize) throws IOException {
   147          LOG.trace("open(Path {}, bufferSize {})", f, bufferSize);
   148          FSDataInputStream lakefsRes = null;
   149          FSDataInputStream s3aRes = null;
   150          IOException lakeFSException = null;
   151          IOException s3aException = null;
   152          Path s3aPath = translateLakeFSPathToS3APath(f);
   153          try {
   154              lakefsRes = lfsFileSystem.open(f, bufferSize);
   155          } catch (IOException e) {
   156              lakeFSException = e;
   157              LOG.error("[open] Can't open {} with lakeFSFileSystem, exception {}", f, e.getMessage());
   158          }
   159          try {
   160              s3aRes = s3AFileSystem.open(translateLakeFSPathToS3APath(f), bufferSize);
   161          } catch (IOException e) {
   162              s3aException = e;
   163              LOG.error("[open] Can't open {} with S3AFileSystem, exception {}", s3aPath, e.getMessage());
   164          }
   165  
   166          if (useLakeFSFileSystemResults && lakeFSException != null) {
   167              LOG.trace("[open] exception by lakeFSFileSystem");
   168              throw lakeFSException;
   169          }
   170          if (!useLakeFSFileSystemResults && s3aException != null) {
   171              LOG.trace("[open] exception by S3AFileSystem");
   172              throw s3aException;
   173          }
   174  
   175          LOG.trace("{}[open] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   176          if (useLakeFSFileSystemResults) {
   177              return lakefsRes;
   178          }
   179          return s3aRes;
   180      }
   181  
   182      @Override
   183      public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize,
   184                                       short replication, long blockSize, Progressable progress) throws IOException {
   185          LOG.trace("create(Path {}, permission {}, overwrite {}, bufferSize {}, replication {}, blockSize {}, progress {})",
   186                  f, permission, overwrite, bufferSize, replication, blockSize, progress);
   187          FSDataOutputStream lakeFSStream = null;
   188          FSDataOutputStream s3aStream = null;
   189          IOException lakeFSException = null;
   190          IOException s3aException = null;
   191          Path s3aPath = translateLakeFSPathToS3APath(f);
   192          try {
   193              lakeFSStream = lfsFileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress);
   194          } catch (IOException e) {
   195              lakeFSException = e;
   196              LOG.error("[create] Can't create {} with lakeFSFileSystem, exception {}", f, e.getMessage());
   197          }
   198          try {
   199              s3aStream = s3AFileSystem.create(s3aPath, permission, overwrite, bufferSize,
   200                  replication, blockSize, progress);
   201          } catch (IOException e) {
   202              s3aException = e;
   203              LOG.error("[create] Can't create {} with S3AFileSystem, exception {}", s3aPath, e.getMessage());
   204          }
   205  
   206          if (useLakeFSFileSystemResults && lakeFSException != null) {
   207              LOG.trace("[create] exception by lakeFSFileSystem");
   208              if (s3aStream != null) {
   209                  s3aStream.close();
   210              }
   211              throw lakeFSException;
   212          }
   213          if (!useLakeFSFileSystemResults && s3aException != null) {
   214              LOG.trace("[create] exception by S3AFileSystem");
   215              if (lakeFSStream != null) {
   216                  lakeFSStream.close();
   217              }
   218              throw s3aException;
   219          }
   220  
   221          LOG.trace("{}[create] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakeFSStream, s3aStream);
   222          TracerOutputTStream tOutputStream = new TracerOutputTStream(lakeFSStream, s3aStream);
   223          return new FSDataOutputStream(tOutputStream, null);
   224      }
   225  
   226      @Override
   227      public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException {
   228          LOG.trace("append(f {}, bufferSize {}, progress {})", f, bufferSize, progress);
   229  
   230          FSDataOutputStream lakefsRes = lfsFileSystem.append(f, bufferSize, progress);
   231          FSDataOutputStream s3aRes = s3AFileSystem.append(translateLakeFSPathToS3APath(f), bufferSize, progress);
   232          LOG.trace("{}[append] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   233  
   234          if (useLakeFSFileSystemResults) {
   235              return lakefsRes;
   236          }
   237          return s3aRes;
   238      }
   239  
   240      @Override
   241      public boolean rename(Path src, Path dst) throws IOException {
   242          LOG.trace("rename(src {}, dst {})", src, dst);
   243          boolean lakefsRes = false;
   244          boolean s3aRes = false;
   245          IOException lakeFSException = null;
   246          IOException s3aException = null;
   247          try {
   248              lakefsRes = lfsFileSystem.rename(src, dst);
   249          } catch (IOException e) {
   250              lakeFSException = e;
   251              LOG.error("[rename] Can't rename {} to {} with lakeFSFileSystem, exception {}", src, dst, e.getMessage());
   252          }
   253          try {
   254              s3aRes = s3AFileSystem.rename(translateLakeFSPathToS3APath(src), translateLakeFSPathToS3APath(dst));
   255          } catch (IOException e) {
   256              s3aException = e;
   257              LOG.error("[rename] Can't rename {} to {} with S3AFileSystem, exception {}", src, dst, e.getMessage());
   258          }
   259  
   260          if (useLakeFSFileSystemResults && lakeFSException != null) {
   261              LOG.trace("[rename] exception by lakeFSFileSystem");
   262              throw lakeFSException;
   263          }
   264          if (!useLakeFSFileSystemResults && s3aException != null) {
   265              LOG.trace("[rename] exception by S3AFileSystem");
   266              throw s3aException;
   267          }
   268  
   269          LOG.trace("{}[rename] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   270          if (useLakeFSFileSystemResults) {
   271              return lakefsRes;
   272          }
   273          return s3aRes;
   274      }
   275  
   276      @Override
   277      public boolean delete(Path f, boolean recursive) throws IOException {
   278          LOG.trace("delete(f {}, recursive {})", f, recursive);
   279          boolean lakefsRes = false;
   280          boolean s3aRes = false;
   281          IOException lakeFSException = null;
   282          IOException s3aException = null;
   283          try {
   284              lakefsRes = delete(f, recursive);
   285          } catch (IOException e) {
   286              lakeFSException = e;
   287              LOG.error("[delete] Can't delete {} with lakeFSFileSystem, exception {}", f, e.getMessage());
   288          }
   289          try {
   290              s3aRes = s3AFileSystem.delete(translateLakeFSPathToS3APath(f), recursive);
   291          } catch (IOException e) {
   292              s3aException = e;
   293              LOG.error("[delete] Can't delete {} to {} with S3AFileSystem, exception {}", f, e.getMessage());
   294          }
   295  
   296          if (useLakeFSFileSystemResults && lakeFSException != null) {
   297              LOG.trace("[delete] exception by lakeFSFileSystem");
   298              throw lakeFSException;
   299          }
   300          if (!useLakeFSFileSystemResults && s3aException != null) {
   301              LOG.trace("[delete] exception by S3AFileSystem");
   302              throw s3aException;
   303          }
   304  
   305          LOG.trace("{}[delete] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   306          if (useLakeFSFileSystemResults) {
   307              return lakefsRes;
   308          }
   309          return s3aRes;
   310      }
   311  
   312      @Override
   313      public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException {
   314          LOG.trace("listStatus(f {})", f);
   315  
   316          FileStatus[] lakefsRes = null;
   317          FileStatus[] s3aRes = null;
   318          IOException lakeFSException = null;
   319          IOException s3aException = null;
   320          Path s3aPath = translateLakeFSPathToS3APath(f);
   321          try {
   322              lakefsRes = lfsFileSystem.listStatus(f);
   323          } catch (IOException e) {
   324              lakeFSException = e;
   325              LOG.error("[listStatus] Can't list the status of {} with lakeFSFileSystem, exception {}", f, e.getMessage());
   326          }
   327          try {
   328              s3aRes = s3AFileSystem.listStatus(s3aPath);
   329          } catch (IOException e) {
   330              s3aException = e;
   331              LOG.error("[listStatus] Can't list the status of {} with S3AFileSystem, exception {}", s3aPath, e.getMessage());
   332          }
   333  
   334          if (useLakeFSFileSystemResults && lakeFSException != null) {
   335              LOG.trace("[listStatus] exception by lakeFSFileSystem");
   336              throw lakeFSException;
   337          }
   338          if (!useLakeFSFileSystemResults && s3aException != null) {
   339              LOG.trace("[listStatus] exception by S3AFileSystem");
   340              throw s3aException;
   341          }
   342  
   343          LOG.trace("{}[listStatus] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   344          if (useLakeFSFileSystemResults) {
   345              return lakefsRes;
   346          }
   347          for (FileStatus stat : s3aRes) {
   348              Path filePath = stat.getPath();
   349              Path lfsPath = translateS3APathToLakeFSPath(filePath);
   350              stat.setPath(lfsPath);
   351          }
   352          return s3aRes;
   353      }
   354  
   355      @Override
   356      public void setWorkingDirectory(Path newDir) {
   357          LOG.trace("setWorkingDirectory(new_dir {})", newDir);
   358  
   359          lfsFileSystem.setWorkingDirectory(newDir);
   360          s3AFileSystem.setWorkingDirectory(translateLakeFSPathToS3APath(newDir));
   361      }
   362  
   363      @Override
   364      public Path getWorkingDirectory() {
   365          LOG.trace("getWorkingDirectory()");
   366  
   367          Path lakefsRes = lfsFileSystem.getWorkingDirectory();
   368          Path s3aRes = s3AFileSystem.getWorkingDirectory();
   369          LOG.trace("{}[getWorkingDirectory] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   370  
   371          if (useLakeFSFileSystemResults) {
   372              return lakefsRes;
   373          }
   374          return s3aRes;
   375      }
   376  
   377      @Override
   378      public boolean mkdirs(Path f, FsPermission permission) throws IOException {
   379          LOG.trace("mkdirs(f {}, permission {})", f, permission);
   380          boolean lakefsRes = false;
   381          boolean s3aRes = false;
   382          IOException lakeFSException = null;
   383          IOException s3aException = null;
   384          try {
   385              lakefsRes = lfsFileSystem.mkdirs(f, permission);
   386          } catch (IOException e) {
   387              lakeFSException = e;
   388              LOG.error("[mkdirs] Can't mkdir {} with lakeFSFileSystem, exception {}", f, e.getMessage());
   389          }
   390          try {
   391              s3aRes = s3AFileSystem.mkdirs(translateLakeFSPathToS3APath(f), permission);
   392          } catch (IOException e) {
   393              s3aException = e;
   394              LOG.error("[mkdirs] Can't mkdir {} to {} with S3AFileSystem, exception {}", f, e.getMessage());
   395          }
   396  
   397          if (useLakeFSFileSystemResults && lakeFSException != null) {
   398              LOG.trace("[mkdirs] exception by lakeFSFileSystem");
   399              throw lakeFSException;
   400          }
   401          if (!useLakeFSFileSystemResults && s3aException != null) {
   402              LOG.trace("[mkdirs] exception by S3AFileSystem");
   403              throw s3aException;
   404          }
   405  
   406          LOG.trace("{}[mkdirs] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   407          if (useLakeFSFileSystemResults) {
   408              return lakefsRes;
   409          }
   410          return s3aRes;
   411      }
   412  
   413      @Override
   414      public FileStatus getFileStatus(Path f) throws IOException {
   415          LOG.trace("getFileStatus(f {})", f);
   416  
   417          FileStatus lakefsRes = null;
   418          FileStatus s3aRes = null;
   419          IOException lakeFSException = null;
   420          IOException s3aException = null;
   421          Path s3aPath = translateLakeFSPathToS3APath(f);
   422          try {
   423              lakefsRes = lfsFileSystem.getFileStatus(f);
   424          } catch (IOException e) {
   425              lakeFSException = e;
   426              LOG.error("[getFileStatus] Can't get {} file status with lakeFSFileSystem, exception {}", f, e.getMessage());
   427          }
   428          try {
   429              s3aRes = s3AFileSystem.getFileStatus(s3aPath);
   430          } catch (IOException e) {
   431              s3aException = e;
   432              LOG.error("[getFileStatus] Can't get {} file status with S3AFileSystem, exception {}", s3aPath, e.getMessage());
   433          }
   434  
   435          if (useLakeFSFileSystemResults && lakeFSException != null) {
   436              LOG.trace("[getFileStatus] exception by lakeFSFileSystem");
   437              throw lakeFSException;
   438          }
   439          if (!useLakeFSFileSystemResults && s3aException != null) {
   440              LOG.trace("[getFileStatus] exception by S3AFileSystem");
   441              throw s3aException;
   442          }
   443  
   444          LOG.trace("{}[getFileStatus] lakefs: {}, s3a: {}", RESULTS_COMPARISON, lakefsRes, s3aRes);
   445          if (useLakeFSFileSystemResults) {
   446              return lakefsRes;
   447          }
   448          Path lfsPath = translateS3APathToLakeFSPath(s3aPath);
   449          s3aRes.setPath(lfsPath);
   450          return s3aRes;
   451      }
   452  
   453      /**
   454       * An output stream encapsulating two output streams, one for each file system the tracer uses.
   455       */
   456      private class TracerOutputTStream extends OutputStream {
   457  
   458          private FSDataOutputStream lakeFSStream;
   459          private FSDataOutputStream s3aStream;
   460  
   461          public TracerOutputTStream(FSDataOutputStream lakeFSStream, FSDataOutputStream s3aStream) throws IOException {
   462              this.lakeFSStream = lakeFSStream;
   463              this.s3aStream = s3aStream;
   464          }
   465  
   466          @Override
   467          public void write(int b) throws IOException {
   468              if (lakeFSStream != null) {
   469                  lakeFSStream.write(b);
   470              }
   471              if (s3aStream != null) {
   472                  s3aStream.write(b);
   473              }
   474          }
   475  
   476          @Override
   477          public void write(@NotNull byte[] b) throws IOException {
   478              if (lakeFSStream != null) {
   479                  lakeFSStream.write(b);
   480              }
   481              if (s3aStream != null) {
   482                  s3aStream.write(b);
   483              }
   484          }
   485  
   486          @Override
   487          public void write(@NotNull byte[] b, int off, int len) throws IOException {
   488              if (lakeFSStream != null) {
   489                  lakeFSStream.write(b, off, len);
   490              }
   491              if (s3aStream != null) {
   492                  s3aStream.write(b, off, len);
   493              }
   494          }
   495  
   496          @Override
   497          public void flush() throws IOException {
   498              if (lakeFSStream != null) {
   499                  lakeFSStream.flush();
   500              }
   501              if (s3aStream != null) {
   502                  s3aStream.flush();
   503              }
   504          }
   505  
   506          @Override
   507          public void close() throws IOException {
   508              if (lakeFSStream != null) {
   509                  lakeFSStream.close();
   510              }
   511              if (s3aStream != null) {
   512                  s3aStream.close();
   513              }
   514          }
   515      }
   516  }