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
    """
    schema = get_schema_from_dataset_url(dataset_url)

    dataset_url_parsed = urlparse(dataset_url)

    resolver = FilesystemResolver(
        dataset_url_parsed,
        spark_session.sparkContext._jsc.hadoopConfiguration())

    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
Example #2
0
def copy_dataset(spark, source_url, target_url, field_regex, not_null_fields,
                 overwrite_output, partitions_count, row_group_size_mb):
    """Creates a copy of a dataset. A new dataset will optionally contain a subset of columns. Rows that have NULL
    values in fields defined by ``not_null_fields`` argument are filtered out.


    :param spark: An instance of ``SparkSession`` object
    :param source_url: A url of the dataset to be copied.
    :param target_url: A url specifying location of the target dataset.
    :param field_regex: A list of regex patterns. Only columns that match one of these patterns are copied to the new
      dataset.
    :param not_null_fields: A list of fields that must have non-NULL valus in the target dataset.
    :param overwrite_output: If ``False`` and there is an existing path defined by ``target_url``, the operation will
      fail.
    :param partitions_count: If not ``None``, the dataset is repartitioned before write. Number of files in the target
      Parquet store is defined by this parameter.
    :param row_group_size_mb: The size of the rowgroup in the target dataset. Specified in megabytes.
    :return: None
    """
    schema = get_schema_from_dataset_url(source_url)

    fields = match_unischema_fields(schema, field_regex)

    if field_regex and not fields:
        field_names = list(schema.fields.keys())
        raise ValueError(
            'Regular expressions (%s) do not match any fields (%s)',
            str(field_regex), str(field_names))

    if fields:
        subschema = schema.create_schema_view(fields)
    else:
        subschema = schema

    with materialize_dataset(spark, target_url, subschema, row_group_size_mb):
        data_frame = spark.read \
            .parquet(source_url)

        if fields:
            data_frame = data_frame.select(*[f.name for f in fields])

        if not_null_fields:
            not_null_condition = reduce(operator.__and__,
                                        (data_frame[f].isNotNull()
                                         for f in not_null_fields))
            data_frame = data_frame.filter(not_null_condition)

        if partitions_count:
            data_frame = data_frame.repartition(partitions_count)

        data_frame.write \
            .mode('overwrite' if overwrite_output else 'error') \
            .option('compression', 'none') \
            .parquet(target_url)
Example #3
0
def make_reader(dataset_url,
                schema_fields=None,
                reader_pool_type='thread',
                workers_count=10,
                pyarrow_serialize=False,
                results_queue_size=50,
                shuffle_row_groups=True,
                shuffle_row_drop_partitions=1,
                predicate=None,
                rowgroup_selector=None,
                num_epochs=1,
                cur_shard=None,
                shard_count=None,
                cache_type='null',
                cache_location=None,
                cache_size_limit=None,
                cache_row_size_estimate=None,
                cache_extra_settings=None,
                hdfs_driver='libhdfs3',
                reader_engine='reader_v1',
                reader_engine_params=None):
    """
    Creates an instance of Reader for reading Petastorm datasets. A Petastorm dataset is a dataset generated using
    :func:`~petastorm.etl.dataset_metadata.materialize_dataset` context manager as explained
    `here <https://petastorm.readthedocs.io/en/latest/readme_include.html#generating-a-dataset>`_.

    See :func:`~petastorm.make_batch_reader` to read from a Parquet store that was not generated using
    :func:`~petastorm.etl.dataset_metadata.materialize_dataset`.

    :param dataset_url: an filepath or a url to a parquet directory,
        e.g. ``'hdfs://some_hdfs_cluster/user/yevgeni/parquet8'``, or ``'file:///tmp/mydataset'``
        or ``'s3://bucket/mydataset'``.
    :param schema_fields: Can be: a list of unischema fields and/or regex pattern strings; ``None`` to read all fields;
            an NGram object, then it will return an NGram of the specified fields.
    :param reader_pool_type: A string denoting the reader pool type. Should be one of ['thread', 'process', 'dummy']
        denoting a thread pool, process pool, or running everything in the master thread. Defaults to 'thread'
    :param workers_count: An int for the number of workers to use in the reader pool. This only is used for the
        thread or process pool. Defaults to 10
    :param pyarrow_serialize: Whether to use pyarrow for serialization. Currently only applicable to process pool.
        Defaults to False.
    :param results_queue_size: Size of the results queue to store prefetched rows. Currently only applicable to
        thread reader pool type.
    :param shuffle_row_groups: Whether to shuffle row groups (the order in which full row groups are read)
    :param shuffle_row_drop_partitions: This is is a positive integer which determines how many partitions to
        break up a row group into for increased shuffling in exchange for worse performance (extra reads).
        For example if you specify 2 each row group read will drop half of the rows within every row group and
        read the remaining rows in separate reads. It is recommended to keep this number below the regular row
        group size in order to not waste reads which drop all rows.
    :param predicate: instance of :class:`.PredicateBase` object to filter rows to be returned by reader. The predicate
        will be passed a single row and must return a boolean value indicating whether to include it in the results.
    :param rowgroup_selector: instance of row group selector object to select row groups to be read
    :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
        ``None`` will result in an infinite number of epochs.
    :param cur_shard: An int denoting the current shard number. Each node reading a shard should
        pass in a unique shard number in the range [0, shard_count). shard_count must be supplied as well.
        Defaults to None
    :param shard_count: An int denoting the number of shards to break this dataset into. Defaults to None
    :param cache_type: A string denoting the cache type, if desired. Options are [None, 'null', 'local-disk'] to
        either have a null/noop cache or a cache implemented using diskcache. Caching is useful when communication
        to the main data store is either slow or expensive and the local machine has large enough storage
        to store entire dataset (or a partition of a dataset if shard_count is used). By default will be a null cache.
    :param cache_location: A string denoting the location or path of the cache.
    :param cache_size_limit: An int specifying the size limit of the cache in bytes
    :param cache_row_size_estimate: An int specifying the estimated size of a row in the dataset
    :param cache_extra_settings: A dictionary of extra settings to pass to the cache implementation,
    :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++)
    :param reader_engine: Multiple engine implementations exist ('reader_v1' and 'experimental_reader_v2'). 'reader_v1'
        (the default value) selects a stable reader implementation.
    :param reader_engine_params: For advanced usage: a dictionary with arguments passed directly to a reader
        implementation constructor chosen by ``reader_engine`` argument.  You should not use this parameter, unless you
        fine-tuning of a reader.
    :return: A :class:`Reader` object
    """

    if dataset_url is None or not isinstance(dataset_url, six.string_types):
        raise ValueError("""dataset_url must be a string""")

    dataset_url = dataset_url[:-1] if dataset_url[-1] == '/' else dataset_url
    logger.debug('dataset_url: %s', dataset_url)

    resolver = FilesystemResolver(dataset_url, hdfs_driver=hdfs_driver)
    filesystem = resolver.filesystem()
    dataset_path = resolver.get_dataset_path()

    if cache_type is None or cache_type == 'null':
        cache = NullCache()
    elif cache_type == 'local-disk':
        cache = LocalDiskCache(cache_location, cache_size_limit,
                               cache_row_size_estimate, **cache_extra_settings
                               or {})
    else:
        raise ValueError('Unknown cache_type: {}'.format(cache_type))

    # Fail if this is a non-petastorm dataset. Typically, a Parquet store will have hundred thousands rows in a single
    # rowgroup. Using PyDictReaderWorker or ReaderV2 implementation is very inefficient as it processes data on a
    # row by row basis. ArrowReaderWorker (used by make_batch_reader) is much more efficient in these cases.
    try:
        dataset_metadata.get_schema_from_dataset_url(dataset_url)
    except PetastormMetadataError:
        raise RuntimeError(
            'Currently make_reader supports reading only Petastorm datasets. '
            'To read from a non-Petastorm Parquet store use make_batch_reader')

    if reader_engine == 'reader_v1':
        if reader_pool_type == 'thread':
            reader_pool = ThreadPool(workers_count, results_queue_size)
        elif reader_pool_type == 'process':
            if pyarrow_serialize:
                serializer = PyArrowSerializer()
            else:
                serializer = PickleSerializer()
            reader_pool = ProcessPool(workers_count, serializer)
        elif reader_pool_type == 'dummy':
            reader_pool = DummyPool()
        else:
            raise ValueError(
                'Unknown reader_pool_type: {}'.format(reader_pool_type))

        # Create a dictionary with all ReaderV2 parameters, so we can merge with reader_engine_params if specified
        kwargs = {
            'schema_fields': schema_fields,
            'reader_pool': reader_pool,
            'shuffle_row_groups': shuffle_row_groups,
            'shuffle_row_drop_partitions': shuffle_row_drop_partitions,
            'predicate': predicate,
            'rowgroup_selector': rowgroup_selector,
            'num_epochs': num_epochs,
            'cur_shard': cur_shard,
            'shard_count': shard_count,
            'cache': cache,
        }

        if reader_engine_params:
            kwargs.update(reader_engine_params)

        try:
            return Reader(filesystem,
                          dataset_path,
                          worker_class=PyDictReaderWorker,
                          **kwargs)
        except PetastormMetadataError as e:
            logger.error('Unexpected exception: %s', str(e))
            raise RuntimeError(
                'make_reader has failed. If you were trying to open a Parquet store that was not '
                'created using Petastorm materialize_dataset and it contains only scalar columns, '
                'you may use make_batch_reader to read it.\n'
                'Inner exception: %s', str(e))

    elif reader_engine == 'experimental_reader_v2':
        if reader_pool_type == 'thread':
            decoder_pool = ThreadPoolExecutor(workers_count)
        elif reader_pool_type == 'process':
            decoder_pool = ProcessPoolExecutor(workers_count)
        elif reader_pool_type == 'dummy':
            decoder_pool = SameThreadExecutor()
        else:
            raise ValueError(
                'Unknown reader_pool_type: {}'.format(reader_pool_type))

        # TODO(yevgeni): once ReaderV2 is ready to be out of experimental status, we should extend
        # the make_reader interfaces to take shuffling buffer parameters explicitly
        shuffling_queue = RandomShufflingBuffer(
            1000, 800) if shuffle_row_groups else NoopShufflingBuffer()

        # Create a dictionary with all ReaderV2 parameters, so we can merge with reader_engine_params if specified
        kwargs = {
            'schema_fields': schema_fields,
            'predicate': predicate,
            'rowgroup_selector': rowgroup_selector,
            'num_epochs': num_epochs,
            'cur_shard': cur_shard,
            'shard_count': shard_count,
            'cache': cache,
            'decoder_pool': decoder_pool,
            'shuffling_queue': shuffling_queue,
            'shuffle_row_groups': shuffle_row_groups,
            'shuffle_row_drop_partitions': shuffle_row_drop_partitions,
        }

        if reader_engine_params:
            kwargs.update(reader_engine_params)

        return ReaderV2(dataset_url, **kwargs)

    else:
        raise ValueError(
            'Unexpected value of reader_engine argument \'%s\'. '
            'Supported reader_engine values are \'reader_v1\' and \'experimental_reader_v2\'',
            reader_engine)
Example #4
0
def reader_v2_throughput(dataset_url, field_regex=None, warmup_cycles_count=300, measure_cycles_count=1000,
                         pool_type=WorkerPoolType.THREAD, loaders_count=3, decoders_count=3,
                         read_method=ReadMethod.PYTHON, shuffling_queue_size=500, min_after_dequeue=400,
                         reader_extra_args=None, pyarrow_serialize=False, spawn_new_process=True):
    """Constructs a ReaderV2 instance and uses it to performs throughput measurements.

    The function will spawn a new process if ``spawn_separate_process`` is set. This is needed to make memory footprint
    measurements accurate.

    :param dataset_url: A url of the dataset to be used for measurements.
    :param field_regex:  A list of regular expressions. Only fields that match one of the regex patterns will be used
      during the benchmark.
    :param warmup_cycles_count: Number of warmup cycles. During warmup cycles no measurements are being recorded.
    :param measure_cycles_count: Number of measurements cycles. Only time elapsed during measurements cycles are used
      in throughput calculations.
    :param pool_type: :class:`WorkerPoolType` enum value.
    :param loaders_count: Number of IO threads.
    :param decoders_count: Number of threads or processes used for decoding. ``pool_type`` parameter defines
      whether multiple processes or threads are used for parallel decoding.
    :param read_method:  An enum :class:`ReadMethod` that defines whether a :class:`petastorm.reader.Reader` will be
      used.
    :param shuffling_queue_size: Maximum number of elements in the shuffling queue.
    :param min_after_dequeue: Minimum number of elements in a shuffling queue before entries can be read from it.
    :param reader_extra_args: Extra arguments that would be passed to Reader constructor.
    :param pyarrow_serialize: When True, pyarrow.serialize library will be used for serializing decoded payloads.
    :param spawn_new_process: This function will respawn itself in a new process if the argument is True. Spawning
      a new process is needed to get an accurate memory footprint.

    :return: An instance of ``BenchmarkResult`` namedtuple with the results of the benchmark. The namedtuple has
      the following fields: `time_mean`, `samples_per_second`, `memory_info` and `cpu`
    """
    if not reader_extra_args:
        reader_extra_args = dict()

    if spawn_new_process:
        args = copy.deepcopy(locals())
        args['spawn_new_process'] = False
        executor = ProcessPoolExecutor(1)
        future = executor.submit(reader_v2_throughput, **args)
        return future.result()

    logger.info('Arguments: %s', locals())

    if 'schema_fields' not in reader_extra_args:
        unischema_fields = match_unischema_fields(get_schema_from_dataset_url(dataset_url), field_regex)
        reader_extra_args['schema_fields'] = unischema_fields

    logger.info('Fields used in the benchmark: %s', str(reader_extra_args['schema_fields']))

    decoder_pool_executor = _create_concurrent_executor(pool_type, decoders_count)

    with ReaderV2(dataset_url, num_epochs=None,
                  loader_pool=ThreadPoolExecutor(loaders_count),
                  decoder_pool=decoder_pool_executor,
                  shuffling_queue=RandomShufflingBuffer(shuffling_queue_size, min_after_dequeue),
                  **reader_extra_args) as reader:

        if read_method == ReadMethod.PYTHON:
            result = _time_warmup_and_work(reader, warmup_cycles_count, measure_cycles_count)
        elif read_method == ReadMethod.TF:
            result = _time_warmup_and_work_tf(reader, warmup_cycles_count, measure_cycles_count, 0, 0)
        else:
            raise RuntimeError('Unexpected reader_type value: %s', str(read_method))

    return result
Example #5
0
def make_reader(dataset_url,
                schema_fields=None,
                reader_pool_type='thread',
                workers_count=10,
                pyarrow_serialize=False,
                results_queue_size=50,
                shuffle_row_groups=True,
                shuffle_row_drop_partitions=1,
                predicate=None,
                rowgroup_selector=None,
                num_epochs=1,
                cur_shard=None,
                shard_count=None,
                shard_seed=None,
                cache_type=NULL_CACHE,
                cache_location=None,
                cache_size_limit=None,
                cache_row_size_estimate=None,
                cache_extra_settings=None,
                hdfs_driver='libhdfs3',
                transform_spec=None,
                filters=None,
                storage_options=None,
                zmq_copy_buffers=True,
                filesystem=None):
    """
    Creates an instance of Reader for reading Petastorm datasets. A Petastorm dataset is a dataset generated using
    :func:`~petastorm.etl.dataset_metadata.materialize_dataset` context manager as explained
    `here <https://petastorm.readthedocs.io/en/latest/readme_include.html#generating-a-dataset>`_.

    See :func:`~petastorm.make_batch_reader` to read from a Parquet store that was not generated using
    :func:`~petastorm.etl.dataset_metadata.materialize_dataset`.

    :param dataset_url: an filepath or a url to a parquet directory,
        e.g. ``'hdfs://some_hdfs_cluster/user/yevgeni/parquet8'``, or ``'file:///tmp/mydataset'``,
        or ``'s3://bucket/mydataset'``, or ``'gs://bucket/mydataset'``.
    :param schema_fields: Can be: a list of unischema fields and/or regex pattern strings; ``None`` to read all fields;
            an NGram object, then it will return an NGram of the specified fields.
    :param reader_pool_type: A string denoting the reader pool type. Should be one of ['thread', 'process', 'dummy']
        denoting a thread pool, process pool, or running everything in the master thread. Defaults to 'thread'
    :param workers_count: An int for the number of workers to use in the reader pool. This only is used for the
        thread or process pool. Defaults to 10
    :param pyarrow_serialize: THE ARGUMENT IS DEPRECATED AND WILL BE REMOVED IN FUTURE VERSIONS.
    :param results_queue_size: Size of the results queue to store prefetched row-groups. Currently only applicable to
        thread reader pool type.
    :param shuffle_row_groups: Whether to shuffle row groups (the order in which full row groups are read)
    :param shuffle_row_drop_partitions: This is is a positive integer which determines how many partitions to
        break up a row group into for increased shuffling in exchange for worse performance (extra reads).
        For example if you specify 2 each row group read will drop half of the rows within every row group and
        read the remaining rows in separate reads. It is recommended to keep this number below the regular row
        group size in order to not waste reads which drop all rows.
    :param predicate: instance of :class:`.PredicateBase` object to filter rows to be returned by reader. The predicate
        will be passed a single row and must return a boolean value indicating whether to include it in the results.
    :param rowgroup_selector: instance of row group selector object to select row groups to be read
    :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
        ``None`` will result in an infinite number of epochs.
    :param cur_shard: An int denoting the current shard number. Each node reading a shard should
        pass in a unique shard number in the range [0, shard_count). shard_count must be supplied as well.
        Defaults to None
    :param shard_count: An int denoting the number of shards to break this dataset into. Defaults to None
    :param shard_seed: Random seed to shuffle row groups for data sharding. Defaults to None
    :param cache_type: A string denoting the cache type, if desired. Options are [None, 'null', 'local-disk'] to
        either have a null/noop cache or a cache implemented using diskcache. Caching is useful when communication
        to the main data store is either slow or expensive and the local machine has large enough storage
        to store entire dataset (or a partition of a dataset if shard_count is used). By default will be a null cache.
    :param cache_location: A string denoting the location or path of the cache.
    :param cache_size_limit: An int specifying the size limit of the cache in bytes
    :param cache_row_size_estimate: An int specifying the estimated size of a row in the dataset
    :param cache_extra_settings: A dictionary of extra settings to pass to the cache implementation,
    :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++)
    :param transform_spec: An instance of :class:`~petastorm.transform.TransformSpec` object defining how a record
        is transformed after it is loaded and decoded. The transformation occurs on a worker thread/process (depends
        on the ``reader_pool_type`` value).
    :param filters: (List[Tuple] or List[List[Tuple]]): Standard PyArrow filters.
        These will be applied when loading the parquet file with PyArrow. More information
        here: https://arrow.apache.org/docs/python/generated/pyarrow.parquet.ParquetDataset.html
    :param storage_options: Dict of kwargs forwarded to ``fsspec`` to initialize the filesystem.
    :param zmq_copy_buffers: A bool indicating whether to use 0mq copy buffers with ProcessPool.
    :param filesystem: An instance of ``pyarrow.FileSystem`` to use. Will ignore storage_options and
        other filesystem configs if it's provided.
    :return: A :class:`Reader` object
    """
    dataset_url = normalize_dir_url(dataset_url)

    filesystem, dataset_path = get_filesystem_and_path_or_paths(
        dataset_url,
        hdfs_driver,
        storage_options=storage_options,
        filesystem=filesystem)

    if cache_type is None or cache_type == NULL_CACHE:
        cache = NullCache()
    elif cache_type == LOCAL_DISK_CACHE:
        cache = LocalDiskCache(cache_location, cache_size_limit,
                               cache_row_size_estimate, **cache_extra_settings
                               or {})
    else:
        raise ValueError('Unknown cache_type: {}'.format(cache_type))

    try:
        dataset_metadata.get_schema_from_dataset_url(
            dataset_url,
            hdfs_driver=hdfs_driver,
            storage_options=storage_options,
            filesystem=filesystem)
    except PetastormMetadataError:
        warnings.warn(
            'Currently make_reader supports reading only Petastorm datasets. '
            'To read from a non-Petastorm Parquet store use make_batch_reader')

    if reader_pool_type == 'thread':
        reader_pool = ThreadPool(workers_count, results_queue_size)
    elif reader_pool_type == 'process':
        if pyarrow_serialize:
            warnings.warn(
                "pyarrow_serializer was deprecated and will be removed in future versions. "
                "The argument no longer has any effect.")
        serializer = PickleSerializer()
        reader_pool = ProcessPool(workers_count,
                                  serializer,
                                  zmq_copy_buffers=zmq_copy_buffers)
    elif reader_pool_type == 'dummy':
        reader_pool = DummyPool()
    else:
        raise ValueError(
            'Unknown reader_pool_type: {}'.format(reader_pool_type))

    kwargs = {
        'schema_fields': schema_fields,
        'reader_pool': reader_pool,
        'shuffle_row_groups': shuffle_row_groups,
        'shuffle_row_drop_partitions': shuffle_row_drop_partitions,
        'predicate': predicate,
        'rowgroup_selector': rowgroup_selector,
        'num_epochs': num_epochs,
        'cur_shard': cur_shard,
        'shard_count': shard_count,
        'shard_seed': shard_seed,
        'cache': cache,
        'transform_spec': transform_spec,
        'filters': filters
    }

    try:
        return Reader(filesystem,
                      dataset_path,
                      worker_class=PyDictReaderWorker,
                      is_batched_reader=False,
                      **kwargs)
    except PetastormMetadataError as e:
        logger.error('Unexpected exception: %s', str(e))
        raise RuntimeError(
            'make_reader has failed. If you were trying to open a Parquet store that was not '
            'created using Petastorm materialize_dataset and it contains only scalar columns, '
            'you may use make_batch_reader to read it.\n'
            'Inner exception: %s', str(e))
Example #6
0
def make_batch_reader(dataset_url_or_urls,
                      schema_fields=None,
                      reader_pool_type='thread',
                      workers_count=10,
                      shuffle_row_groups=True,
                      shuffle_row_drop_partitions=1,
                      predicate=None,
                      rowgroup_selector=None,
                      num_epochs=1,
                      cur_shard=None,
                      shard_count=None,
                      shard_seed=None,
                      cache_type='null',
                      cache_location=None,
                      cache_size_limit=None,
                      cache_row_size_estimate=None,
                      cache_extra_settings=None,
                      hdfs_driver='libhdfs3',
                      transform_spec=None,
                      filters=None,
                      storage_options=None,
                      zmq_copy_buffers=True,
                      filesystem=None):
    """
    Creates an instance of Reader for reading batches out of a non-Petastorm Parquet store.

    Currently, only stores having native scalar parquet data types are supported.
    Use :func:`~petastorm.make_reader` to read Petastorm Parquet stores generated with
    :func:`~petastorm.etl.dataset_metadata.materialize_dataset`.

    NOTE: only scalar columns or array type (of primitive type element) columns are currently supported.

    NOTE: If without `schema_fields` specified, the reader schema will be inferred from parquet dataset. then the
    reader schema fields order will preserve parqeut dataset fields order (partition column come first), but if
    setting `transform_spec` and specified `TransformSpec.selected_fields`, then the reader schema fields order
    will be the order of 'selected_fields'.

    :param dataset_url_or_urls: a url to a parquet directory or a url list (with the same scheme) to parquet files.
        e.g. ``'hdfs://some_hdfs_cluster/user/yevgeni/parquet8'``, or ``'file:///tmp/mydataset'``,
        or ``'s3://bucket/mydataset'``, or ``'gs://bucket/mydataset'``,
        or ``[file:///tmp/mydataset/00000.parquet, file:///tmp/mydataset/00001.parquet]``.
    :param schema_fields: A list of regex pattern strings. Only columns matching at least one of the
        patterns in the list will be loaded.
    :param reader_pool_type: A string denoting the reader pool type. Should be one of ['thread', 'process', 'dummy']
        denoting a thread pool, process pool, or running everything in the master thread. Defaults to 'thread'
    :param workers_count: An int for the number of workers to use in the reader pool. This only is used for the
        thread or process pool. Defaults to 10
    :param shuffle_row_groups: Whether to shuffle row groups (the order in which full row groups are read)
    :param shuffle_row_drop_partitions: This is is a positive integer which determines how many partitions to
        break up a row group into for increased shuffling in exchange for worse performance (extra reads).
        For example if you specify 2 each row group read will drop half of the rows within every row group and
        read the remaining rows in separate reads. It is recommended to keep this number below the regular row
        group size in order to not waste reads which drop all rows.
    :param predicate: instance of :class:`.PredicateBase` object to filter rows to be returned by reader. The predicate
        will be passed a pandas DataFrame object and must return a pandas Series with boolean values of matching
        dimensions.
    :param rowgroup_selector: instance of row group selector object to select row groups to be read
    :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
        ``None`` will result in an infinite number of epochs.
    :param cur_shard: An int denoting the current shard number. Each node reading a shard should
        pass in a unique shard number in the range [0, shard_count). shard_count must be supplied as well.
        Defaults to None
    :param shard_count: An int denoting the number of shards to break this dataset into. Defaults to None
    :param shard_seed: Random seed to shuffle row groups for data sharding. Defaults to None
    :param cache_type: A string denoting the cache type, if desired. Options are [None, 'null', 'local-disk'] to
        either have a null/noop cache or a cache implemented using diskcache. Caching is useful when communication
        to the main data store is either slow or expensive and the local machine has large enough storage
        to store entire dataset (or a partition of a dataset if shard_count is used). By default will be a null cache.
    :param cache_location: A string denoting the location or path of the cache.
    :param cache_size_limit: An int specifying the size limit of the cache in bytes
    :param cache_row_size_estimate: An int specifying the estimated size of a row in the dataset
    :param cache_extra_settings: A dictionary of extra settings to pass to the cache implementation,
    :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++)
    :param transform_spec: An instance of :class:`~petastorm.transform.TransformSpec` object defining how a record
        is transformed after it is loaded and decoded. The transformation occurs on a worker thread/process (depends
        on the ``reader_pool_type`` value).
    :param filters: (List[Tuple] or List[List[Tuple]]): Standard PyArrow filters.
        These will be applied when loading the parquet file with PyArrow. More information
        here: https://arrow.apache.org/docs/python/generated/pyarrow.parquet.ParquetDataset.html
    :param storage_options: Dict of kwargs forwarded to ``fsspec`` to initialize the filesystem.
    :param zmq_copy_buffers: A bool indicating whether to use 0mq copy buffers with ProcessPool.
    :param filesystem: An instance of ``pyarrow.FileSystem`` to use. Will ignore storage_options and
        other filesystem configs if it's provided.
    :return: A :class:`Reader` object
    """
    dataset_url_or_urls = normalize_dataset_url_or_urls(dataset_url_or_urls)

    filesystem, dataset_path_or_paths = get_filesystem_and_path_or_paths(
        dataset_url_or_urls,
        hdfs_driver,
        storage_options=storage_options,
        filesystem=filesystem)

    try:
        dataset_metadata.get_schema_from_dataset_url(
            dataset_url_or_urls,
            hdfs_driver=hdfs_driver,
            storage_options=storage_options,
            filesystem=filesystem)
        warnings.warn(
            'Please use make_reader (instead of \'make_batch_dataset\' function to read this dataset. '
            'You may get unexpected results. '
            'Currently make_batch_reader supports reading only Parquet stores that contain '
            'standard Parquet data types and do not require petastorm decoding.'
        )
    except PetastormMetadataError:
        pass

    if cache_type is None or cache_type == NULL_CACHE:
        cache = NullCache()
    elif cache_type == LOCAL_DISK_CACHE:
        cache = LocalDiskArrowTableCache(cache_location, cache_size_limit,
                                         cache_row_size_estimate,
                                         **cache_extra_settings or {})
    else:
        raise ValueError('Unknown cache_type: {}'.format(cache_type))

    if reader_pool_type == 'thread':
        reader_pool = ThreadPool(workers_count)
    elif reader_pool_type == 'process':
        serializer = ArrowTableSerializer()
        reader_pool = ProcessPool(workers_count,
                                  serializer,
                                  zmq_copy_buffers=zmq_copy_buffers)
    elif reader_pool_type == 'dummy':
        reader_pool = DummyPool()
    else:
        raise ValueError(
            'Unknown reader_pool_type: {}'.format(reader_pool_type))

    return Reader(filesystem,
                  dataset_path_or_paths,
                  schema_fields=schema_fields,
                  worker_class=ArrowReaderWorker,
                  reader_pool=reader_pool,
                  shuffle_row_groups=shuffle_row_groups,
                  shuffle_row_drop_partitions=shuffle_row_drop_partitions,
                  predicate=predicate,
                  rowgroup_selector=rowgroup_selector,
                  num_epochs=num_epochs,
                  cur_shard=cur_shard,
                  shard_count=shard_count,
                  shard_seed=shard_seed,
                  cache=cache,
                  transform_spec=transform_spec,
                  is_batched_reader=True,
                  filters=filters)
Example #7
0
def test_get_schema_from_dataset_url_bogus_url():
    with pytest.raises(IOError):
        get_schema_from_dataset_url('file:///non-existing-path')

    with pytest.raises(ValueError):
        get_schema_from_dataset_url('/invalid_url')
Example #8
0
def test_get_schema_from_dataset_url(synthetic_dataset):
    schema = get_schema_from_dataset_url(synthetic_dataset.url)
    assert TestSchema.fields == schema.fields
Example #9
0
def make_batch_reader(dataset_url,
                      schema_fields=None,
                      reader_pool_type='thread', workers_count=10,
                      shuffle_row_groups=True, shuffle_row_drop_partitions=1,
                      predicate=None,
                      rowgroup_selector=None,
                      num_epochs=1,
                      cur_shard=None, shard_count=None,
                      cache_type='null', cache_location=None, cache_size_limit=None,
                      cache_row_size_estimate=None, cache_extra_settings=None,
                      hdfs_driver='libhdfs3',
                      transform_spec=None):
    """
    Creates an instance of Reader for reading batches out of a non-Petastorm Parquet store.

    Currently, only stores having native scalar parquet data types are supported.
    Use :func:`~petastorm.make_reader` to read Petastorm Parquet stores generated with
    :func:`~petastorm.etl.dataset_metadata.materialize_dataset`.

    NOTE: only scalar columns are currently supported.

    :param dataset_url: an filepath or a url to a parquet directory,
        e.g. ``'hdfs://some_hdfs_cluster/user/yevgeni/parquet8'``, or ``'file:///tmp/mydataset'``
        or ``'s3://bucket/mydataset'``.
    :param schema_fields: A list of regex pattern strings. Only columns matching at least one of the
        patterns in the list will be loaded.
    :param reader_pool_type: A string denoting the reader pool type. Should be one of ['thread', 'process', 'dummy']
        denoting a thread pool, process pool, or running everything in the master thread. Defaults to 'thread'
    :param workers_count: An int for the number of workers to use in the reader pool. This only is used for the
        thread or process pool. Defaults to 10
    :param shuffle_row_groups: Whether to shuffle row groups (the order in which full row groups are read)
    :param shuffle_row_drop_partitions: This is is a positive integer which determines how many partitions to
        break up a row group into for increased shuffling in exchange for worse performance (extra reads).
        For example if you specify 2 each row group read will drop half of the rows within every row group and
        read the remaining rows in separate reads. It is recommended to keep this number below the regular row
        group size in order to not waste reads which drop all rows.
    :param predicate: instance of :class:`.PredicateBase` object to filter rows to be returned by reader. The predicate
        will be passed a pandas DataFrame object and must return a pandas Series with boolean values of matching
        dimensions.
    :param rowgroup_selector: instance of row group selector object to select row groups to be read
    :param num_epochs: An epoch is a single pass over all rows in the dataset. Setting ``num_epochs`` to
        ``None`` will result in an infinite number of epochs.
    :param cur_shard: An int denoting the current shard number. Each node reading a shard should
        pass in a unique shard number in the range [0, shard_count). shard_count must be supplied as well.
        Defaults to None
    :param shard_count: An int denoting the number of shards to break this dataset into. Defaults to None
    :param cache_type: A string denoting the cache type, if desired. Options are [None, 'null', 'local-disk'] to
        either have a null/noop cache or a cache implemented using diskcache. Caching is useful when communication
        to the main data store is either slow or expensive and the local machine has large enough storage
        to store entire dataset (or a partition of a dataset if shard_count is used). By default will be a null cache.
    :param cache_location: A string denoting the location or path of the cache.
    :param cache_size_limit: An int specifying the size limit of the cache in bytes
    :param cache_row_size_estimate: An int specifying the estimated size of a row in the dataset
    :param cache_extra_settings: A dictionary of extra settings to pass to the cache implementation,
    :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++)
    :param transform_spec: An instance of :class:`~petastorm.transform.TransformSpec` object defining how a record
        is transformed after it is loaded and decoded. The transformation occurs on a worker thread/process (depends
        on the ``reader_pool_type`` value).
    :return: A :class:`Reader` object
    """

    if dataset_url is None or not isinstance(dataset_url, six.string_types):
        raise ValueError('dataset_url must be a string')

    try:
        dataset_metadata.get_schema_from_dataset_url(dataset_url, hdfs_driver=hdfs_driver)
        warnings.warn('Please use make_reader (instead of \'make_batch_dataset\' function to read this dataset. '
                      'You may get unexpected results. '
                      'Currently make_batch_reader supports reading only Parquet stores that contain '
                      'standard Parquet data types and do not require petastorm decoding.')
    except PetastormMetadataError:
        pass

    dataset_url = dataset_url[:-1] if dataset_url[-1] == '/' else dataset_url
    logger.debug('dataset_url: %s', dataset_url)

    resolver = FilesystemResolver(dataset_url, hdfs_driver=hdfs_driver)
    filesystem = resolver.filesystem()
    dataset_path = resolver.get_dataset_path()

    if cache_type is None or cache_type == 'null':
        cache = NullCache()
    elif cache_type == 'local-disk':
        cache = LocalDiskArrowTableCache(cache_location, cache_size_limit, cache_row_size_estimate,
                                         **cache_extra_settings or {})
    else:
        raise ValueError('Unknown cache_type: {}'.format(cache_type))

    if reader_pool_type == 'thread':
        reader_pool = ThreadPool(workers_count)
    elif reader_pool_type == 'process':
        serializer = ArrowTableSerializer()
        reader_pool = ProcessPool(workers_count, serializer)
    elif reader_pool_type == 'dummy':
        reader_pool = DummyPool()
    else:
        raise ValueError('Unknown reader_pool_type: {}'.format(reader_pool_type))

    return Reader(filesystem, dataset_path,
                  schema_fields=schema_fields,
                  worker_class=ArrowReaderWorker,
                  reader_pool=reader_pool,
                  shuffle_row_groups=shuffle_row_groups,
                  shuffle_row_drop_partitions=shuffle_row_drop_partitions,
                  predicate=predicate,
                  rowgroup_selector=rowgroup_selector,
                  num_epochs=num_epochs,
                  cur_shard=cur_shard,
                  shard_count=shard_count,
                  cache=cache,
                  transform_spec=transform_spec)
Example #10
0
def copy_dataset(spark,
                 source_url,
                 target_url,
                 field_regex,
                 not_null_fields,
                 overwrite_output,
                 partitions_count,
                 row_group_size_mb,
                 hdfs_driver='libhdfs3'):
    """
    Creates a copy of a dataset. A new dataset will optionally contain a subset of columns. Rows that have NULL
    values in fields defined by ``not_null_fields`` argument are filtered out.


    :param spark: An instance of ``SparkSession`` object
    :param source_url: A url of the dataset to be copied.
    :param target_url: A url specifying location of the target dataset.
    :param field_regex: A list of regex patterns. Only columns that match one of these patterns are copied to the new
      dataset.
    :param not_null_fields: A list of fields that must have non-NULL valus in the target dataset.
    :param overwrite_output: If ``False`` and there is an existing path defined by ``target_url``, the operation will
      fail.
    :param partitions_count: If not ``None``, the dataset is repartitioned before write. Number of files in the target
      Parquet store is defined by this parameter.
    :param row_group_size_mb: The size of the rowgroup in the target dataset. Specified in megabytes.
    :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++)
    :param user: String denoting username when connecting to HDFS. None implies login user.
    :return: None
    """
    schema = get_schema_from_dataset_url(source_url, hdfs_driver=hdfs_driver)

    fields = match_unischema_fields(schema, field_regex)

    if field_regex and not fields:
        field_names = list(schema.fields.keys())
        raise ValueError(
            'Regular expressions (%s) do not match any fields (%s)',
            str(field_regex), str(field_names))

    if fields:
        subschema = schema.create_schema_view(fields)
    else:
        subschema = schema

    resolver = FilesystemResolver(
        target_url,
        spark.sparkContext._jsc.hadoopConfiguration(),
        hdfs_driver=hdfs_driver,
        user=spark.sparkContext.sparkUser())
    with materialize_dataset(spark,
                             target_url,
                             subschema,
                             row_group_size_mb,
                             filesystem_factory=resolver.filesystem_factory()):
        data_frame = spark.read \
            .parquet(source_url)

        if fields:
            data_frame = data_frame.select(*[f.name for f in fields])

        if not_null_fields:
            not_null_condition = reduce(operator.__and__,
                                        (data_frame[f].isNotNull()
                                         for f in not_null_fields))
            data_frame = data_frame.filter(not_null_condition)

        if partitions_count:
            data_frame = data_frame.repartition(partitions_count)

        data_frame.write \
            .mode('overwrite' if overwrite_output else 'error') \
            .option('compression', 'none') \
            .parquet(target_url)