Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 1 addition & 3 deletions petastorm/reader.py
Original file line number Diff line number Diff line change
Expand Up @@ -532,9 +532,7 @@ def _apply_predicate_to_row_groups(self, dataset, row_groups, predicate):
raise ValueError('predicate parameter is expected to be derived from PredicateBase')
predicate_fields = predicate.get_fields()

if set(predicate_fields) == dataset.partitions.partition_names:
assert len(dataset.partitions.partition_names) == 1, \
'Datasets with only a single partition level supported at the moment'
if set(predicate_fields).issubset(dataset.partitions.partition_names):

filtered_row_group_indexes = []
for piece_index, piece in enumerate(row_groups):
Expand Down
6 changes: 5 additions & 1 deletion petastorm/tests/test_predicates.py
Original file line number Diff line number Diff line change
Expand Up @@ -154,12 +154,14 @@ def test_predicate_on_partitioned_dataset(tmpdir):
"""
TestSchema = Unischema('TestSchema', [
UnischemaField('id', np.int32, (), ScalarCodec(IntegerType()), False),
UnischemaField('id2', np.int32, (), ScalarCodec(IntegerType()), False),
UnischemaField('test_field', np.int32, (), ScalarCodec(IntegerType()), False),
])

def test_row_generator(x):
"""Returns a single entry in the generated dataset."""
return {'id': x,
'id2': x+1,
'test_field': x*x}

rowgroup_size_mb = 256
Expand All @@ -177,11 +179,13 @@ def test_row_generator(x):

spark.createDataFrame(rows_rdd, TestSchema.as_spark_schema()) \
.write \
.partitionBy('id') \
.partitionBy('id', 'id2') \
.parquet(dataset_url)

with make_reader(dataset_url, predicate=in_lambda(['id'], lambda x: x == 3)) as reader:
assert next(reader).id == 3
with make_reader(dataset_url, predicate=in_lambda(['id2'], lambda x: x == 5)) as reader:
assert next(reader).id == 5
with make_reader(dataset_url, predicate=in_lambda(['id'], lambda x: x == '3')) as reader:
with pytest.raises(StopIteration):
# Predicate should have selected none, so a StopIteration should be raised.
Expand Down