Skip to main content

SharingClient

The SharingClient class provides methods to explore and discover shared data from a Delta Sharing server.

Constructor

SharingClient(profile: Union[str, BinaryIO, TextIO, Path, DeltaSharingProfile])
profile
Union[str, BinaryIO, TextIO, Path, DeltaSharingProfile]
required
Path to the profile file, file-like object, or DeltaSharingProfile instance.Supported formats:
  • Local file path: /path/to/profile.share
  • FSSPEC URL: s3://bucket/profile.share
  • DBFS path: /dbfs/path/to/profile.share
  • File-like object (BinaryIO or TextIO)
  • DeltaSharingProfile instance
Example:
import delta_sharing

# From file path
client = delta_sharing.SharingClient("/path/to/profile.share")

# From S3
client = delta_sharing.SharingClient("s3://my-bucket/profile.share")

# From DeltaSharingProfile object
from delta_sharing.protocol import DeltaSharingProfile
profile = DeltaSharingProfile.read_from_file("/path/to/profile.share")
client = delta_sharing.SharingClient(profile)

Methods

list_shares()

List all shares accessible to you in the Delta Sharing server.
list_shares() -> Sequence[Share]
Returns: A sequence of Share objects. Example:
shares = client.list_shares()
for share in shares:
    print(share.name)

list_schemas()

List all schemas in a specific share.
list_schemas(share: Share) -> Sequence[Schema]
share
Share
required
The share object to list schemas from.
Returns: A sequence of Schema objects. Example:
from delta_sharing.protocol import Share

share = Share(name="my_share")
schemas = client.list_schemas(share)
for schema in schemas:
    print(f"{schema.share}.{schema.name}")

list_tables()

List all tables in a specific schema.
list_tables(schema: Schema) -> Sequence[Table]
schema
Schema
required
The schema object to list tables from.
Returns: A sequence of Table objects. Example:
from delta_sharing.protocol import Schema

schema = Schema(name="my_schema", share="my_share")
tables = client.list_tables(schema)
for table in tables:
    print(f"{table.share}.{table.schema}.{table.name}")

list_all_tables()

List all tables accessible to you across all shares and schemas.
list_all_tables() -> Sequence[Table]
Returns: A sequence of Table objects. Example:
tables = client.list_all_tables()
for table in tables:
    print(f"{table.share}.{table.schema}.{table.name}")
This method may fallback to iterating through shares and schemas if the server doesn’t support the all-tables API.

Data Loading Functions

load_as_pandas()

Load a shared table as a pandas DataFrame.
load_as_pandas(
    url: str,
    limit: Optional[int] = None,
    version: Optional[int] = None,
    timestamp: Optional[str] = None,
    jsonPredicateHints: Optional[str] = None,
    use_delta_format: Optional[bool] = None,
    convert_in_batches: bool = False,
) -> pd.DataFrame
url
str
required
Table URL in the format <profile-path>#<share>.<schema>.<table>
limit
int
default:"None"
Load only the specified number of rows. Useful for sampling large tables. Must be non-negative.
version
int
default:"None"
Load the table snapshot at a specific version. Must be non-negative. Cannot be used with timestamp.
timestamp
str
default:"None"
Load the table snapshot at or after this timestamp. Format: YYYY-MM-DDThh:mm:ssZ. Cannot be used with version.
jsonPredicateHints
str
default:"None"
JSON string containing predicate hints for filtering. See Advanced Usage for details.
use_delta_format
bool
default:"None"
Whether to use Delta format or Parquet format. If not specified, determined from table metadata.
convert_in_batches
bool
default:"False"
Whether to convert files to pandas one batch at a time to reduce memory usage.
Returns: A pandas DataFrame. Examples:
import delta_sharing

table_url = "/path/to/profile.share#my_share.my_schema.my_table"
df = delta_sharing.load_as_pandas(table_url)

load_as_spark()

Load a shared table as an Apache Spark DataFrame.
Requires PySpark to be installed and the Apache Spark Connector for Delta Sharing to be configured.
load_as_spark(
    url: str,
    version: Optional[int] = None,
    timestamp: Optional[str] = None,
    delta_sharing_profile: Optional[DeltaSharingProfile] = None,
) -> PySparkDataFrame
url
str
required
Table URL. Format depends on delta_sharing_profile:
  • With profile parameter: <share>.<schema>.<table>
  • Without profile parameter: <profile-path>#<share>.<schema>.<table>
version
int
default:"None"
Load the table snapshot at a specific version. Cannot be used with timestamp.
timestamp
str
default:"None"
Load the table snapshot at this timestamp. Format: YYYY-MM-DDThh:mm:ssZ. Cannot be used with version.
delta_sharing_profile
DeltaSharingProfile
default:"None"
Profile object for authentication. If provided, credentials are passed as Spark options instead of reading from a file.
Returns: A Spark DataFrame. Examples:
import delta_sharing

table_url = "/path/to/profile.share#my_share.my_schema.my_table"
df = delta_sharing.load_as_spark(table_url)
df.show()

load_table_changes_as_pandas()

Load table changes (Change Data Feed) as a pandas DataFrame.
load_table_changes_as_pandas(
    url: str,
    starting_version: Optional[int] = None,
    ending_version: Optional[int] = None,
    starting_timestamp: Optional[str] = None,
    ending_timestamp: Optional[str] = None,
    use_delta_format: Optional[bool] = None,
    convert_in_batches: bool = False,
) -> pd.DataFrame
url
str
required
Table URL in the format <profile-path>#<share>.<schema>.<table>
starting_version
int
default:"None"
Starting version (inclusive). Either this or starting_timestamp is required.
ending_version
int
default:"None"
Ending version (inclusive). If not provided, uses the latest version.
starting_timestamp
str
default:"None"
Starting timestamp. Format: YYYY-MM-DDThh:mm:ssZ. Either this or starting_version is required.
ending_timestamp
str
default:"None"
Ending timestamp (inclusive). Format: YYYY-MM-DDThh:mm:ssZ. If not provided, uses the latest version.
use_delta_format
bool
default:"None"
Whether to use Delta format. Defaults to Parquet format if not specified.
convert_in_batches
bool
default:"False"
Convert files to pandas one batch at a time to reduce memory usage.
Returns: A pandas DataFrame with columns:
  • All original table columns
  • _change_type: insert, update_preimage, update_postimage, or delete
  • _commit_version: Version number of the change
  • _commit_timestamp: Timestamp of the change
Examples:
# Get changes from version 0 to 5
changes = delta_sharing.load_table_changes_as_pandas(
    table_url,
    starting_version=0,
    ending_version=5
)

load_table_changes_as_spark()

Load table changes (Change Data Feed) as a Spark DataFrame.
load_table_changes_as_spark(
    url: str,
    starting_version: Optional[int] = None,
    ending_version: Optional[int] = None,
    starting_timestamp: Optional[str] = None,
    ending_timestamp: Optional[str] = None,
    delta_sharing_profile: Optional[DeltaSharingProfile] = None,
) -> PySparkDataFrame
url
str
required
Table URL. Format depends on delta_sharing_profile:
  • With profile parameter: <share>.<schema>.<table>
  • Without profile parameter: <profile-path>#<share>.<schema>.<table>
starting_version
int
default:"None"
Starting version (inclusive). Either this or starting_timestamp is required.
ending_version
int
default:"None"
Ending version (inclusive). Uses latest if not provided.
starting_timestamp
str
default:"None"
Starting timestamp. Format: YYYY-MM-DDThh:mm:ssZ.
ending_timestamp
str
default:"None"
Ending timestamp. Format: YYYY-MM-DDThh:mm:ssZ.
delta_sharing_profile
DeltaSharingProfile
default:"None"
Profile object for authentication.
Returns: A Spark DataFrame. Example:
changes = delta_sharing.load_table_changes_as_spark(
    table_url,
    starting_version=0,
    ending_version=5
)
changes.show()

Metadata Functions

get_table_version()

Get the current or historical version of a shared table.
get_table_version(
    url: str,
    starting_timestamp: Optional[str] = None
) -> int
url
str
required
Table URL in the format <profile-path>#<share>.<schema>.<table>
starting_timestamp
str
default:"None"
Get the version at or after this timestamp. Format: YYYY-MM-DDThh:mm:ssZ. Returns latest version if not specified.
Returns: The table version as an integer. Examples:
# Get current version
version = delta_sharing.get_table_version(table_url)
print(f"Current version: {version}")

# Get version at timestamp
version = delta_sharing.get_table_version(
    table_url,
    starting_timestamp="2024-01-15T10:00:00Z"
)

get_table_metadata()

Get the metadata of a shared table.
get_table_metadata(
    url: str,
    use_delta_format: bool = True
) -> Metadata
url
str
required
Table URL in the format <profile-path>#<share>.<schema>.<table>
use_delta_format
bool
default:"True"
Whether to request Delta format metadata.
Returns: A Metadata object containing:
  • id: Table UUID
  • name: Table name
  • description: Table description
  • format: Format provider (e.g., “parquet”)
  • schema_string: JSON string representation of the schema
  • configuration: Table configuration
  • partition_columns: List of partition column names
Example:
metadata = delta_sharing.get_table_metadata(table_url)
print(f"Table ID: {metadata.id}")
print(f"Schema: {metadata.schema_string}")
print(f"Partition columns: {metadata.partition_columns}")

get_table_protocol()

Get the protocol version of a shared table.
get_table_protocol(
    url: str,
    use_delta_format: bool = True
) -> Protocol
url
str
required
Table URL in the format <profile-path>#<share>.<schema>.<table>
use_delta_format
bool
default:"True"
Whether to request Delta format protocol.
Returns: A Protocol object containing:
  • min_reader_version: Minimum reader version required
  • min_writer_version: Minimum writer version required (optional)
  • reader_features: List of reader features (optional)
Example:
protocol = delta_sharing.get_table_protocol(table_url)
print(f"Min reader version: {protocol.min_reader_version}")
if protocol.reader_features:
    print(f"Reader features: {protocol.reader_features}")

Data Classes

Share

Represents a share in Delta Sharing.
@dataclass(frozen=True)
class Share:
    name: str

Schema

Represents a schema within a share.
@dataclass(frozen=True)
class Schema:
    name: str
    share: str

Table

Represents a table within a schema.
@dataclass(frozen=True)
class Table:
    name: str
    share: str
    schema: str

DeltaSharingProfile

Represents authentication credentials for a Delta Sharing server.
@dataclass(frozen=True)
class DeltaSharingProfile:
    share_credentials_version: int
    endpoint: str
    bearer_token: Optional[str] = None
    expiration_time: Optional[str] = None
    type: Optional[str] = None
    token_endpoint: Optional[str] = None
    client_id: Optional[str] = None
    client_secret: Optional[str] = None
    # ... additional OAuth fields
Example:
from delta_sharing.protocol import DeltaSharingProfile

profile = DeltaSharingProfile.read_from_file("/path/to/profile.share")
print(f"Endpoint: {profile.endpoint}")
print(f"Version: {profile.share_credentials_version}")

Build docs developers (and LLMs) love