Skip to content

Commit

Permalink
[AIR] Make PathPartitionScheme a dataclass (#28390)
Browse files Browse the repository at this point in the history
Signed-off-by: Balaji Veeramani <bveeramani@berkeley.edu>
  • Loading branch information
bveeramani authored Sep 14, 2022
1 parent 10996bd commit c91b4a7
Showing 1 changed file with 25 additions and 48 deletions.
73 changes: 25 additions & 48 deletions python/ray/data/datasource/partitioning.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
from dataclasses import dataclass
import posixpath
from enum import Enum
from typing import (
Expand Down Expand Up @@ -35,67 +36,43 @@ class PartitionStyle(str, Enum):


@DeveloperAPI
@dataclass
class Partitioning:
"""Partition scheme used to describe path-based partitions.
Path-based partition formats embed all partition keys and values directly in
their dataset file paths.
Attributes:
style: The partition style - may be either HIVE or DIRECTORY.
base_dir: "/"-delimited base directory that all partitioned paths should
exist under (exclusive). File paths either outside of, or at the first
level of, this directory will be considered unpartitioned. Specify
`None` or an empty string to search for partitions in all file path
directories.
field_names: The partition key field names (i.e. column names for tabular
datasets). When non-empty, the order and length of partition key
field names must match the order and length of partition values.
Required when parsing DIRECTORY partitioned paths or generating
HIVE partitioned paths.
filesystem: Filesystem that will be used for partition path file I/O.
"""

def __init__(
self,
style: PartitionStyle,
base_dir: Optional[str] = None,
field_names: Optional[List[str]] = None,
filesystem: Optional["pyarrow.fs.FileSystem"] = None,
):
"""Creates a new path-based dataset partition scheme.
style: PartitionStyle
base_dir: Optional[str] = None
field_names: Optional[List[str]] = None
filesystem: Optional["pyarrow.fs.FileSystem"] = None

Args:
style: The partition style - may be either HIVE or DIRECTORY.
base_dir: "/"-delimited base directory that all partitioned paths should
exist under (exclusive). File paths either outside of, or at the first
level of, this directory will be considered unpartitioned. Specify
`None` or an empty string to search for partitions in all file path
directories.
field_names: The partition key field names (i.e. column names for tabular
datasets). When non-empty, the order and length of partition key
field names must match the order and length of partition values.
Required when parsing DIRECTORY partitioned paths or generating
HIVE partitioned paths.
filesystem: Filesystem that will be used for partition path file I/O.
"""
self._style = style
self._base_dir = base_dir or ""
self._field_names = field_names
self._filesystem = filesystem
def __post_init__(self):
if self.base_dir is None:
self.base_dir = ""
self._normalize_base_dir()

@property
def style(self) -> PartitionStyle:
"""Gets the path partitioning style."""
return self._style

@property
def base_dir(self) -> str:
"""Gets the original base directory supplied during object construction."""
return self._base_dir

@property
def normalized_base_dir(self) -> str:
"""Returns the base directory normalized for compatibility with a filesystem."""
return self._normalized_base_dir

@property
def field_names(self) -> Optional[List[str]]:
"""Gets the partition key field names."""
return self._field_names

@property
def filesystem(self) -> Optional["pyarrow.fs.FileSystem"]:
"""Gets the original filesystem supplied during object construction."""
return self._filesystem

@property
def resolved_filesystem(self) -> "pyarrow.fs.FileSystem":
"""Returns the filesystem resolved for compatibility with a base directory."""
Expand All @@ -114,8 +91,8 @@ def _normalize_base_dir(self):
)

paths, self._resolved_filesystem = _resolve_paths_and_filesystem(
self._base_dir,
self._filesystem,
self.base_dir,
self.filesystem,
)
assert (
len(paths) == 1
Expand Down

0 comments on commit c91b4a7

Please sign in to comment.