def _load_rows_with_predicate(self, pq_file, piece, worker_predicate, shuffle_row_drop_partition): """Loads all rows that match a predicate from a piece""" # 1. Read all columns needed by predicate and decode # 2. Apply the predicate. If nothing matches, exit early # 3. Read the remaining columns and decode # 4. Combine with columns already decoded for the predicate. # Split all column names into ones that are needed by predicateand the rest. predicate_column_names = set(worker_predicate.get_fields()) if not predicate_column_names: raise ValueError('At least one field name must be returned by predicate\'s get_field() method') all_schema_names = set(field.name for field in self._schema.fields.values()) invalid_column_names = predicate_column_names - all_schema_names if invalid_column_names: raise ValueError('At least some column names requested by the predicate ({}) ' 'are not valid schema names: ({})'.format(', '.join(invalid_column_names), ', '.join(all_schema_names))) other_column_names = all_schema_names - predicate_column_names - \ self._dataset.partitions.partition_names # Read columns needed for the predicate predicate_rows = self._read_with_shuffle_row_drop(piece, pq_file, predicate_column_names, shuffle_row_drop_partition) # Decode values decoded_predicate_rows = [utils.decode_row(_select_cols(row, predicate_column_names), self._schema) for row in predicate_rows] # Use the predicate to filter match_predicate_mask = [worker_predicate.do_include(row) for row in decoded_predicate_rows] # Don't have anything left after filtering? Exit early. if not any(match_predicate_mask): return [] # Remove rows that were filtered out by the predicate filtered_decoded_predicate_rows = [row for i, row in enumerate(decoded_predicate_rows) if match_predicate_mask[i]] if other_column_names: # Read remaining columns other_rows = self._read_with_shuffle_row_drop(piece, pq_file, other_column_names, shuffle_row_drop_partition) # Remove rows that were filtered out by the predicate filtered_other_rows = [row for i, row in enumerate(other_rows) if match_predicate_mask[i]] # Decode remaining columns decoded_other_rows = [utils.decode_row(row, self._schema) for row in filtered_other_rows] # Merge predicate needed columns with the remaining all_cols = [_merge_two_dicts(a, b) for a, b in zip(decoded_other_rows, filtered_decoded_predicate_rows)] return all_cols else: return filtered_decoded_predicate_rows
def decode(self, row): """Decodes each value in the dictionary as defined by Unischema (passed to the constructor). Returns a namedtuple Example (ngram is None): Input: row = { 'field1': bytearray(), ... } Output: ( field1=numpy.array(...), ... ) Example (ngram): row = { -1 : { 'field1': bytearray(), ...}, 0 : { 'field2': bytearray(), ...}, 1 : { 'field1': bytearray(), ...}, } Output: row = { -1 : ( field1=numpy.array(...), ...), 0 : ( field2=numpy.array(...), ...), 1 : ( field1=numpy.array(...), ...), } :param row: A dictionary of fields :return: A named tuple (or a dictionary of namedtuples in case of a non None ngram) """ if self._ngram: for key in row.keys(): current_schema = self._ngram.get_schema_at_timestep( self._schema, key) row[key] = current_schema.make_namedtuple( **utils.decode_row(row[key], current_schema)) return row else: return self._schema.make_namedtuple( **utils.decode_row(row, self._schema))
def dataset_as_rdd(dataset_url, spark_session, schema_fields=None): """ Retrieve a spark rdd for a given petastorm dataset :param dataset_url: A string for the dataset url (e.g. hdfs:///path/to/dataset) :param spark_session: A spark session :param schema_fields: list of unischema fields to subset, or None to read all fields. :return: A rdd of dictionary records from the dataset """ dataset_url_parsed = urlparse(dataset_url) resolver = FilesystemResolver( dataset_url_parsed, spark_session.sparkContext._jsc.hadoopConfiguration()) dataset = pq.ParquetDataset(resolver.parsed_dataset_url().path, filesystem=resolver.filesystem(), validate_schema=False) schema = dataset_metadata.get_schema(dataset) dataset_df = spark_session.read.parquet(resolver.parsed_dataset_url().path) if schema_fields is not None: # If wanting a subset of fields, create the schema view and run a select on those fields schema = schema.create_schema_view(schema_fields) field_names = [field.name for field in schema_fields] dataset_df = dataset_df.select(*field_names) dataset_rows = dataset_df.rdd\ .map(lambda row: utils.decode_row(row.asDict(), schema))\ .map(lambda record: schema.make_namedtuple(**record)) return dataset_rows
def dataset_as_rdd(dataset_url, spark_session, schema_fields=None, hdfs_driver='libhdfs3'): """ Retrieve a spark rdd for a given petastorm dataset :param dataset_url: A string for the dataset url (e.g. hdfs:///path/to/dataset) :param spark_session: A spark session :param schema_fields: list of unischema fields to subset, or None to read all fields. :param hdfs_driver: A string denoting the hdfs driver to use (if using a dataset on hdfs). Current choices are libhdfs (java through JNI) or libhdfs3 (C++) :return: A rdd of dictionary records from the dataset """ schema = get_schema_from_dataset_url(dataset_url, hdfs_driver=hdfs_driver) dataset_url_parsed = urlparse(dataset_url) resolver = FilesystemResolver( dataset_url_parsed, spark_session.sparkContext._jsc.hadoopConfiguration(), hdfs_driver=hdfs_driver) dataset_df = spark_session.read.parquet(resolver.get_dataset_path()) if schema_fields is not None: # If wanting a subset of fields, create the schema view and run a select on those fields schema = schema.create_schema_view(schema_fields) field_names = [field.name for field in schema_fields] dataset_df = dataset_df.select(*field_names) dataset_rows = dataset_df.rdd \ .map(lambda row: utils.decode_row(row.asDict(), schema)) \ .map(lambda record: schema.make_namedtuple(**record)) return dataset_rows
def test_decode_decimal_scalar_when_codec_is_none(): """Decoding a row that has a field with the codec set to None. The type should be deduced automatically from UnischemaField's numpy_dtype attribute if the type is either a numpy scalar or a Decimal""" MatrixSchema = Unischema('TestSchema', [UnischemaField('scalar', Decimal, ())]) row = {'scalar': '123.45'} decoded_value = decode_row(row, MatrixSchema)['scalar'] assert decoded_value == Decimal('123.45') assert isinstance(decoded_value, Decimal) row = {'scalar': Decimal('123.45')} decoded_value = decode_row(row, MatrixSchema)['scalar'] assert decoded_value == Decimal('123.45') assert isinstance(decoded_value, Decimal)
def test_nominal_case(): """Nominal flow: can decode field successfully""" expected = np.random.rand(10, 10) row = {'matrix': NdarrayCodec().encode(MatrixField, expected)} actual = decode_row(row, MatrixSchema)['matrix'] np.testing.assert_equal(actual, expected)
def test_decode_numpy_scalar_with_unknown_dtype(): """If numpy_dtype is None, then the value is not decoded, just passed through.""" MatrixSchema = Unischema('TestSchema', [UnischemaField('scalar', None, ())]) row = {'scalar': [4, 2]} decoded_value = decode_row(row, MatrixSchema)['scalar'] assert decoded_value == [4, 2]
def test_decode_numpy_scalar_when_codec_is_none(): """Decoding a row that has a field with the codec set to None. The type should be deduced automatically from UnischemaField's numpy_dtype attribute""" MatrixSchema = Unischema('TestSchema', [UnischemaField('scalar', np.float64, ())]) row = {'scalar': 42.0} decoded_value = decode_row(row, MatrixSchema)['scalar'] assert decoded_value == 42 assert isinstance(decoded_value, np.float64)
def _load_rows(self, pq_file, piece, shuffle_row_drop_range): """Loads all rows from a piece""" # pyarrow would fail if we request a column names that the dataset is partitioned by, so we strip them from # the `columns` argument. partitions = self._dataset.partitions column_names = set(field.name for field in self._schema.fields.values()) - partitions.partition_names all_rows = self._read_with_shuffle_row_drop(piece, pq_file, column_names, shuffle_row_drop_range) return [utils.decode_row(row, self._schema) for row in all_rows]
def test_decode_numpy_scalar_with_explicit_scalar_codec(): """Decoding a row that has a field with the codec set explicitly""" MatrixSchema = Unischema('TestSchema', [ UnischemaField('scalar', np.float64, (), ScalarCodec(DoubleType()), False) ]) row = {'scalar': 42.0} decoded_value = decode_row(row, MatrixSchema)['scalar'] assert decoded_value == 42 assert isinstance(decoded_value, np.float64)
def _index_columns(piece_info, dataset_url, partitions, indexers, schema, hdfs_driver='libhdfs3'): """ Function build indexes for dataset piece described in piece_info :param piece_info: description of dataset piece :param dataset_url: dataset location :param partitions: dataset partitions :param indexers: list of indexer objects :param schema: dataset schema :param hdfs_driver: A string denoting the hdfs driver to use (if using a dataset on hdfs). Current choices are libhdfs (java through JNI) or libhdfs3 (C++) :return: list of indexers containing index data """ # Resolver in executor context will get hadoop config from environment resolver = FilesystemResolver(dataset_url, hdfs_driver=hdfs_driver) fs = resolver.filesystem() # Create pyarrow piece piece = pq.ParquetDatasetPiece(piece_info.path, open_file_func=fs.open, row_group=piece_info.row_group, partition_keys=piece_info.partition_keys) # Collect column names needed for indexing column_names = set() for indexer in indexers: column_names.update(indexer.column_names) # Read columns needed for indexing column_rows = piece.read( columns=list(column_names), partitions=partitions).to_pandas().to_dict('records') # Decode columns values decoded_rows = [utils.decode_row(row, schema) for row in column_rows] if not decoded_rows: raise ValueError( 'Cannot build index with empty decoded_rows, columns: {}, partitions: {}' .format(column_names, partitions)) # Index columns values for indexer in indexers: indexer.build_index(decoded_rows, piece_info.piece_index) # Indexer objects contain index data, it will be consolidated on reduce phace return indexers
def _load_rows(self, piece, shuffle_row_drop_range): """Loads all rows from a piece""" # pyarrow would fail if we request a column names that the dataset is partitioned by, so we strip them from # the `columns` argument. column_names = list(field.name for field in self._schema.fields.values()) all_rows = self._read_with_shuffle_row_drop(piece, column_names, shuffle_row_drop_range) transform_func = self._transform_spec.func if self._transform_spec else ( lambda x: x) return [ transform_func(utils.decode_row(row, self._schema)) for row in all_rows ]
def _load_rows(self, pq_file, piece, shuffle_row_drop_range): """Loads all rows from a piece""" # pyarrow would fail if we request a column names that the dataset is partitioned by, so we strip them from # the `columns` argument. partitions = self._dataset.partitions # self._dataset.partitions is None, if make_reader is created directly from a parquet file # (and not a directory with *.parquet files) partition_names = partitions.partition_names if partitions else set() column_names = set(field.name for field in self._schema.fields.values()) - partition_names all_rows = self._read_with_shuffle_row_drop(piece, pq_file, column_names, shuffle_row_drop_range) all_rows = [utils.decode_row(row, self._schema) for row in all_rows] if self._transform_spec: all_rows = _apply_transform_spec(all_rows, self._transform_spec) return all_rows
def test_can_not_decode(): """Make sure field name is part of the error message""" row = {'matrix': 'bogus'} with pytest.raises(DecodeFieldError, match='matrix'): decode_row(row, MatrixSchema)