Packages

trait LogStore extends AnyRef

General interface for all critical file system operations required to read and write the DeltaLog. The correctness of the DeltaLog is predicated on the atomicity and durability guarantees of the implementation of this interface. Specifically,

1. Atomic visibility of files: Any file written through this store must be made visible atomically. In other words, this should not generate partial files.

2. Mutual exclusion: Only one writer must be able to create (or rename) a file at the final destination.

3. Consistent listing: Once a file has been written in a directory, all future listings for that directory must return that file.

Linear Supertypes
AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. LogStore
  2. AnyRef
  3. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. All

Abstract Value Members

  1. abstract def invalidateCache(): Unit

    Invalidate any caching that the implementation may be using

  2. abstract def listFrom(path: Path): Iterator[FileStatus]

    List the paths in the same directory that are lexicographically greater or equal to (UTF-8 sorting) the given path.

    List the paths in the same directory that are lexicographically greater or equal to (UTF-8 sorting) the given path. The result should also be sorted by the file name.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  3. abstract def read(path: Path): Seq[String]

    Load the given file and return a Seq of lines.

    Load the given file and return a Seq of lines. The line break will be removed from each line. This method will load the entire file into the memory. Call readAsIterator if possible as its implementation may be more efficient.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  4. abstract def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit

    Write the given actions to the given path with or without overwrite as indicated.

    Write the given actions to the given path with or without overwrite as indicated. Implementation must throw java.nio.file.FileAlreadyExistsException exception if the file already exists and overwrite = false. Furthermore, implementation must ensure that the entire file is made visible atomically, that is, it should not generate partial files.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

Concrete Value Members

  1. final def !=(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  2. final def ##(): Int
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  4. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  5. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native()
  6. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  7. def equals(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  8. def finalize(): Unit
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  9. final def getClass(): Class[_]
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  10. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  11. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  12. def isPartialWriteVisible(path: Path, hadoopConf: Configuration): Boolean

    Whether a partial write is visible when writing to path.

    Whether a partial write is visible when writing to path.

    As this depends on the underlying file system implementations, we require the input of path here in order to identify the underlying file system, even though in most cases a log store only deals with one file system.

    The default value is only provided here for legacy reasons, which will be removed. Any LogStore implementation should override this instead of relying on the default.

    Note: The default implementation ignores the hadoopConf parameter to provide the backward compatibility. Subclasses should override this method and use hadoopConf properly to support passing Hadoop file system configurations through DataFrame options.

  13. def listFrom(path: Path, hadoopConf: Configuration): Iterator[FileStatus]

    List the paths in the same directory that are lexicographically greater or equal to (UTF-8 sorting) the given path.

    List the paths in the same directory that are lexicographically greater or equal to (UTF-8 sorting) the given path. The result should also be sorted by the file name.

    Note: The default implementation ignores the hadoopConf parameter to provide the backward compatibility. Subclasses should override this method and use hadoopConf properly to support passing Hadoop file system configurations through DataFrame options.

  14. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  15. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  16. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  17. final def read(fileStatus: FileStatus, hadoopConf: Configuration): Seq[String]

    Load the given file represented by fileStatus and return a Seq of lines.

    Load the given file represented by fileStatus and return a Seq of lines. The line break will be removed from each line.

    Note: Using a stale FileStatus may get an incorrect result.

  18. def read(path: Path, hadoopConf: Configuration): Seq[String]

    Load the given file and return a Seq of lines.

    Load the given file and return a Seq of lines. The line break will be removed from each line. This method will load the entire file into the memory. Call readAsIterator if possible as its implementation may be more efficient.

    Note: The default implementation ignores the hadoopConf parameter to provide the backward compatibility. Subclasses should override this method and use hadoopConf properly to support passing Hadoop file system configurations through DataFrame options.

  19. def readAsIterator(fileStatus: FileStatus, hadoopConf: Configuration): ClosableIterator[String]

    Load the file represented by given fileStatus and return an iterator of lines.

    Load the file represented by given fileStatus and return an iterator of lines. The line break will be removed from each line.

    Note-1: the returned ClosableIterator should be closed when it's no longer used to avoid resource leak.

    Note-2: Using a stale FileStatus may get an incorrect result.

  20. def readAsIterator(path: Path, hadoopConf: Configuration): ClosableIterator[String]

    Load the given file and return an iterator of lines.

    Load the given file and return an iterator of lines. The line break will be removed from each line. The default implementation calls read to load the entire file into the memory. An implementation should provide a more efficient approach if possible. For example, the file content can be loaded on demand.

    Note: the returned ClosableIterator should be closed when it's no longer used to avoid resource leak.

    Note: The default implementation ignores the hadoopConf parameter to provide the backward compatibility. Subclasses should override this method and use hadoopConf properly to support passing Hadoop file system configurations through DataFrame options.

  21. def resolvePathOnPhysicalStorage(path: Path, hadoopConf: Configuration): Path

    Resolve the fully qualified path for the given path.

    Resolve the fully qualified path for the given path.

    Note: The default implementation ignores the hadoopConf parameter to provide the backward compatibility. Subclasses should override this method and use hadoopConf properly to support passing Hadoop file system configurations through DataFrame options.

  22. final def synchronized[T0](arg0: ⇒ T0): T0
    Definition Classes
    AnyRef
  23. def toString(): String
    Definition Classes
    AnyRef → Any
  24. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  25. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  26. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native()
  27. def write(path: Path, actions: Iterator[String], overwrite: Boolean, hadoopConf: Configuration): Unit

    Write the given actions to the given path with or without overwrite as indicated.

    Write the given actions to the given path with or without overwrite as indicated. Implementation must throw java.nio.file.FileAlreadyExistsException exception if the file already exists and overwrite = false. Furthermore, implementation must ensure that the entire file is made visible atomically, that is, it should not generate partial files.

    Note: The default implementation ignores the hadoopConf parameter to provide the backward compatibility. Subclasses should override this method and use hadoopConf properly to support passing Hadoop file system configurations through DataFrame options.

Deprecated Value Members

  1. def isPartialWriteVisible(path: Path): Boolean

    Whether a partial write is visible when writing to path.

    Whether a partial write is visible when writing to path.

    As this depends on the underlying file system implementations, we require the input of path here in order to identify the underlying file system, even though in most cases a log store only deals with one file system.

    The default value is only provided here for legacy reasons, which will be removed. Any LogStore implementation should override this instead of relying on the default.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  2. final def listFrom(path: String): Iterator[FileStatus]

    List the paths in the same directory that are lexicographically greater or equal to (UTF-8 sorting) the given path.

    List the paths in the same directory that are lexicographically greater or equal to (UTF-8 sorting) the given path. The result should also be sorted by the file name.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  3. final def read(path: String): Seq[String]

    Load the given file and return a Seq of lines.

    Load the given file and return a Seq of lines. The line break will be removed from each line. This method will load the entire file into the memory. Call readAsIterator if possible as its implementation may be more efficient.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  4. def readAsIterator(path: Path): ClosableIterator[String]

    Load the given file and return an iterator of lines.

    Load the given file and return an iterator of lines. The line break will be removed from each line. The default implementation calls read to load the entire file into the memory. An implementation should provide a more efficient approach if possible. For example, the file content can be loaded on demand.

    Note: the returned ClosableIterator should be closed when it's no longer used to avoid resource leak.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  5. final def readAsIterator(path: String): ClosableIterator[String]

    Load the given file and return an iterator of lines.

    Load the given file and return an iterator of lines. The line break will be removed from each line. The default implementation calls read to load the entire file into the memory. An implementation should provide a more efficient approach if possible. For example, the file content can be loaded on demand.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  6. def resolvePathOnPhysicalStorage(path: Path): Path

    Resolve the fully qualified path for the given path.

    Resolve the fully qualified path for the given path.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

  7. final def write(path: String, actions: Iterator[String]): Unit

    Write the given actions to the given path without overwriting any existing file.

    Write the given actions to the given path without overwriting any existing file. Implementation must throw java.nio.file.FileAlreadyExistsException exception if the file already exists. Furthermore, implementation must ensure that the entire file is made visible atomically, that is, it should not generate partial files.

    Annotations
    @deprecated
    Deprecated

    call the method that asks for a Hadoop Configuration object instead

Inherited from AnyRef

Inherited from Any

Ungrouped