Packages

c

org.apache.spark.sql.connector.catalog

DelegatingCatalogExtension

abstract class DelegatingCatalogExtension extends CatalogExtension

A simple implementation of CatalogExtension, which implements all the catalog functions by calling the built-in session catalog directly. This is created for convenience, so that users only need to override some methods where they want to apply custom logic. For example, they can override createTable, do something else before calling super.createTable.

Annotations
@Evolving()
Source
DelegatingCatalogExtension.java
Since

3.0.0

Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. DelegatingCatalogExtension
  2. CatalogExtension
  3. SupportsNamespaces
  4. FunctionCatalog
  5. TableCatalog
  6. CatalogPlugin
  7. AnyRef
  8. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. Protected

Instance Constructors

  1. new DelegatingCatalogExtension()

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. def alterNamespace(namespace: Array[String], changes: <repeated...>[NamespaceChange]): Unit

    Apply a set of metadata changes to a namespace in the catalog.

    Apply a set of metadata changes to a namespace in the catalog.

    namespace

    a multi-part namespace

    changes

    a collection of changes to apply to the namespace

    Definition Classes
    DelegatingCatalogExtensionSupportsNamespaces
    Annotations
    @Override()
    Exceptions thrown

    NoSuchNamespaceException If the namespace does not exist (optional)

    UnsupportedOperationException If namespace properties are not supported

  5. def alterTable(ident: Identifier, changes: <repeated...>[TableChange]): Table

    Apply a set of changes to a table in the catalog.

    Apply a set of changes to a table in the catalog.

    Implementations may reject the requested changes. If any change is rejected, none of the changes should be applied to the table.

    The requested changes must be applied in the order given.

    If the catalog supports views and contains a view for the identifier and not a table, this must throw NoSuchTableException.

    ident

    a table identifier

    changes

    changes to apply to the table

    returns

    updated metadata for the table. This can be null if getting the metadata for the updated table is expensive. Spark always discard the returned table here.

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Exceptions thrown

    IllegalArgumentException If any change is rejected by the implementation.

    NoSuchTableException If the table doesn't exist or is a view

  6. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  7. def capabilities(): Set[TableCatalogCapability]

    returns

    the set of capabilities for this TableCatalog

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
  8. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.CloneNotSupportedException]) @IntrinsicCandidate() @native()
  9. def createNamespace(namespace: Array[String], metadata: Map[String, String]): Unit

    Create a namespace in the catalog.

    Create a namespace in the catalog.

    namespace

    a multi-part namespace

    metadata

    a string map of properties for the given namespace

    Definition Classes
    DelegatingCatalogExtensionSupportsNamespaces
    Annotations
    @Override()
    Exceptions thrown

    NamespaceAlreadyExistsException If the namespace already exists

    UnsupportedOperationException If create is not a supported operation

  10. def createTable(ident: Identifier, columns: Array[Column], partitions: Array[Transform], properties: Map[String, String]): Table

    Create a table in the catalog.

    Create a table in the catalog.

    ident

    a table identifier

    columns

    the columns of the new table.

    partitions

    transforms to use for partitioning data in the table

    properties

    a string map of table properties

    returns

    metadata for the new table. This can be null if getting the metadata for the new table is expensive. Spark will call #loadTable(Identifier) if needed (e.g. CTAS).

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchNamespaceException If the identifier namespace does not exist (optional)

    TableAlreadyExistsException If a table or view already exists for the identifier

    UnsupportedOperationException If a requested partition transform is not supported

  11. def defaultNamespace(): Array[String]

    Return a default namespace for the catalog.

    Return a default namespace for the catalog.

    When this catalog is set as the current catalog, the namespace returned by this method will be set as the current namespace.

    The namespace returned by this method is not required to exist.

    returns

    a multi-part namespace

    Definition Classes
    DelegatingCatalogExtensionCatalogPlugin
    Annotations
    @Override()
  12. def dropNamespace(namespace: Array[String], cascade: Boolean): Boolean

    Drop a namespace from the catalog with cascade mode, recursively dropping all objects within the namespace if cascade is true.

    Drop a namespace from the catalog with cascade mode, recursively dropping all objects within the namespace if cascade is true.

    If the catalog implementation does not support this operation, it may throw UnsupportedOperationException.

    namespace

    a multi-part namespace

    cascade

    When true, deletes all objects under the namespace

    returns

    true if the namespace was dropped

    Definition Classes
    DelegatingCatalogExtensionSupportsNamespaces
    Annotations
    @Override()
    Exceptions thrown

    NoSuchNamespaceException If the namespace does not exist (optional)

    NonEmptyNamespaceException If the namespace is non-empty and cascade is false

    UnsupportedOperationException If drop is not a supported operation

  13. def dropTable(ident: Identifier): Boolean

    Drop a table in the catalog.

    Drop a table in the catalog.

    If the catalog supports views and contains a view for the identifier and not a table, this must not drop the view and must return false.

    ident

    a table identifier

    returns

    true if a table was deleted, false if no table exists for the identifier

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
  14. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  15. def equals(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef → Any
  16. def functionExists(ident: Identifier): Boolean

    Returns true if the function exists, false otherwise.

    Returns true if the function exists, false otherwise.

    Definition Classes
    DelegatingCatalogExtensionFunctionCatalog
    Annotations
    @Override()
    Since

    3.3.0

  17. final def getClass(): Class[_ <: AnyRef]
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  18. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  19. def initialize(name: String, options: CaseInsensitiveStringMap): Unit

    Called to initialize configuration.

    Called to initialize configuration.

    This method is called once, just after the provider is instantiated.

    name

    the name used to identify and load this catalog

    options

    a case-insensitive string map of configuration

    Definition Classes
    DelegatingCatalogExtensionCatalogPlugin
    Annotations
    @Override()
  20. def invalidateTable(ident: Identifier): Unit

    Invalidate cached table metadata for an identifier.

    Invalidate cached table metadata for an identifier.

    If the table is already loaded or cached, drop cached data. If the table does not exist or is not cached, do nothing. Calling this method should not query remote services.

    ident

    a table identifier

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
  21. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  22. def listFunctions(namespace: Array[String]): Array[Identifier]

    List the functions in a namespace from the catalog.

    List the functions in a namespace from the catalog.

    If there are no functions in the namespace, implementations should return an empty array.

    namespace

    a multi-part namespace

    returns

    an array of Identifiers for functions

    Definition Classes
    DelegatingCatalogExtensionFunctionCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchNamespaceException If the namespace does not exist (optional).

  23. def listNamespaces(namespace: Array[String]): Array[Array[String]]

    List namespaces in a namespace.

    List namespaces in a namespace.

    If an object such as a table, view, or function exists, its parent namespaces must also exist and must be returned by this discovery method. For example, if table a.b.t exists, this method invoked as listNamespaces(["a"]) must return ["a", "b"] in the result array.

    namespace

    a multi-part namespace

    returns

    an array of multi-part namespace names

    Definition Classes
    DelegatingCatalogExtensionSupportsNamespaces
    Annotations
    @Override()
    Exceptions thrown

    NoSuchNamespaceException If the namespace does not exist (optional)

  24. def listNamespaces(): Array[Array[String]]

    List top-level namespaces from the catalog.

    List top-level namespaces from the catalog.

    If an object such as a table, view, or function exists, its parent namespaces must also exist and must be returned by this discovery method. For example, if table a.b.t exists, this method must return ["a"] in the result array.

    returns

    an array of multi-part namespace names

    Definition Classes
    DelegatingCatalogExtensionSupportsNamespaces
    Annotations
    @Override()
  25. def listTables(namespace: Array[String]): Array[Identifier]

    List the tables in a namespace from the catalog.

    List the tables in a namespace from the catalog.

    If the catalog supports views, this must return identifiers for only tables and not views.

    namespace

    a multi-part namespace

    returns

    an array of Identifiers for tables

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchNamespaceException If the namespace does not exist (optional).

  26. def loadFunction(ident: Identifier): UnboundFunction

    Load a function by identifier from the catalog.

    Load a function by identifier from the catalog.

    ident

    a function identifier

    returns

    an unbound function instance

    Definition Classes
    DelegatingCatalogExtensionFunctionCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchFunctionException If the function doesn't exist

  27. def loadNamespaceMetadata(namespace: Array[String]): Map[String, String]

    Load metadata properties for a namespace.

    Load metadata properties for a namespace.

    namespace

    a multi-part namespace

    returns

    a string map of properties for the given namespace

    Definition Classes
    DelegatingCatalogExtensionSupportsNamespaces
    Annotations
    @Override()
    Exceptions thrown

    NoSuchNamespaceException If the namespace does not exist (optional)

    UnsupportedOperationException If namespace properties are not supported

  28. def loadTable(ident: Identifier, version: String): Table

    Load table metadata of a specific version by identifier from the catalog.

    Load table metadata of a specific version by identifier from the catalog.

    If the catalog supports views and contains a view for the identifier and not a table, this must throw NoSuchTableException.

    ident

    a table identifier

    version

    version of the table

    returns

    the table's metadata

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchTableException If the table doesn't exist or is a view

  29. def loadTable(ident: Identifier, timestamp: Long): Table

    Load table metadata at a specific time by identifier from the catalog.

    Load table metadata at a specific time by identifier from the catalog.

    If the catalog supports views and contains a view for the identifier and not a table, this must throw NoSuchTableException.

    ident

    a table identifier

    timestamp

    timestamp of the table, which is microseconds since 1970-01-01 00:00:00 UTC

    returns

    the table's metadata

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchTableException If the table doesn't exist or is a view

  30. def loadTable(ident: Identifier): Table

    Load table metadata by identifier from the catalog.

    Load table metadata by identifier from the catalog.

    If the catalog supports views and contains a view for the identifier and not a table, this must throw NoSuchTableException.

    ident

    a table identifier

    returns

    the table's metadata

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchTableException If the table doesn't exist or is a view

  31. def loadTable(ident: Identifier, writePrivileges: Set[TableWritePrivilege]): Table

    Load table metadata by identifier from the catalog.

    Load table metadata by identifier from the catalog. Spark will write data into this table later.

    If the catalog supports views and contains a view for the identifier and not a table, this must throw NoSuchTableException.

    ident

    a table identifier

    returns

    the table's metadata

    Definition Classes
    TableCatalog
    Since

    3.5.3

    Exceptions thrown

    NoSuchTableException If the table doesn't exist or is a view

  32. def name(): String

    Called to get this catalog's name.

    Called to get this catalog's name.

    This method is only called after CaseInsensitiveStringMap) is called to pass the catalog's name.

    Definition Classes
    DelegatingCatalogExtensionCatalogPlugin
    Annotations
    @Override()
  33. def namespaceExists(namespace: Array[String]): Boolean

    Test whether a namespace exists.

    Test whether a namespace exists.

    If an object such as a table, view, or function exists, its parent namespaces must also exist. For example, if table a.b.t exists, this method invoked as namespaceExists(["a"]) or namespaceExists(["a", "b"]) must return true.

    namespace

    a multi-part namespace

    returns

    true if the namespace exists, false otherwise

    Definition Classes
    DelegatingCatalogExtensionSupportsNamespaces
    Annotations
    @Override()
  34. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  35. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  36. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  37. def purgeTable(ident: Identifier): Boolean

    Drop a table in the catalog and completely remove its data by skipping a trash even if it is supported.

    Drop a table in the catalog and completely remove its data by skipping a trash even if it is supported.

    If the catalog supports views and contains a view for the identifier and not a table, this must not drop the view and must return false.

    If the catalog supports to purge a table, this method should be overridden. The default implementation throws UnsupportedOperationException.

    ident

    a table identifier

    returns

    true if a table was deleted, false if no table exists for the identifier

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Since

    3.1.0

    Exceptions thrown

    UnsupportedOperationException If table purging is not supported

  38. def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit

    Renames a table in the catalog.

    Renames a table in the catalog.

    If the catalog supports views and contains a view for the old identifier and not a table, this throws NoSuchTableException. Additionally, if the new identifier is a table or a view, this throws TableAlreadyExistsException.

    If the catalog does not support table renames between namespaces, it throws UnsupportedOperationException.

    oldIdent

    the table identifier of the existing table to rename

    newIdent

    the new table identifier of the table

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Exceptions thrown

    NoSuchTableException If the table to rename doesn't exist or is a view

    TableAlreadyExistsException If the new table name already exists or is a view

    UnsupportedOperationException If the namespaces of old and new identifiers do not match (optional)

  39. final def setDelegateCatalog(delegate: CatalogPlugin): Unit

    This will be called only once by Spark to pass in the Spark built-in session catalog, after CaseInsensitiveStringMap) is called.

    This will be called only once by Spark to pass in the Spark built-in session catalog, after CaseInsensitiveStringMap) is called.

    Definition Classes
    DelegatingCatalogExtensionCatalogExtension
    Annotations
    @Override()
  40. final def synchronized[T0](arg0: => T0): T0
    Definition Classes
    AnyRef
  41. def tableExists(ident: Identifier): Boolean

    Test whether a table exists using an identifier from the catalog.

    Test whether a table exists using an identifier from the catalog.

    If the catalog supports views and contains a view for the identifier and not a table, this must return false.

    ident

    a table identifier

    returns

    true if the table exists, false otherwise

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
  42. def toString(): String
    Definition Classes
    AnyRef → Any
  43. def useNullableQuerySchema(): Boolean

    If true, mark all the fields of the query schema as nullable when executing CREATE/REPLACE TABLE ...

    If true, mark all the fields of the query schema as nullable when executing CREATE/REPLACE TABLE ... AS SELECT ... and creating the table.

    Definition Classes
    TableCatalog
  44. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])
  45. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException]) @native()
  46. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])

Deprecated Value Members

  1. def createTable(ident: Identifier, schema: StructType, partitions: Array[Transform], properties: Map[String, String]): Table

    Create a table in the catalog.

    Create a table in the catalog.

    Definition Classes
    DelegatingCatalogExtensionTableCatalog
    Annotations
    @Override()
    Deprecated

    This is deprecated. Please override Column[], Transform[], Map) instead.

  2. def finalize(): Unit
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.Throwable]) @Deprecated
    Deprecated

    (Since version 9)

Inherited from CatalogExtension

Inherited from SupportsNamespaces

Inherited from FunctionCatalog

Inherited from TableCatalog

Inherited from CatalogPlugin

Inherited from AnyRef

Inherited from Any

Ungrouped