Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[DataFrame] Adding read methods and tests #1712

Merged
merged 24 commits into from
Apr 21, 2018
Merged
Changes from 1 commit
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
Prev Previous commit
Next Next commit
restoring distributed read csv
  • Loading branch information
adgirish authored and kunalgosar committed Apr 19, 2018
commit a3dcbc638ddf0205b4a285b0bcda16306eaba3c1
131 changes: 71 additions & 60 deletions python/ray/dataframe/io.py
Original file line number Diff line number Diff line change
Expand Up @@ -192,66 +192,77 @@ def read_csv(filepath,
kwargs: Keyword arguments in pandas::from_csv
"""

warnings.warn("Defaulting to Pandas implementation",
PendingDeprecationWarning)

port_frame = pd.read_csv(filepath,
sep=',',
delimiter=None,
header='infer',
names=None,
index_col=None,
usecols=None,
squeeze=False,
prefix=None,
mangle_dupe_cols=True,
dtype=None,
engine=None,
converters=None,
true_values=None,
false_values=None,
skipinitialspace=False,
skiprows=None,
nrows=None,
na_values=None,
keep_default_na=True,
na_filter=True,
verbose=False,
skip_blank_lines=True,
parse_dates=False,
infer_datetime_format=False,
keep_date_col=False,
date_parser=None,
dayfirst=False,
iterator=False,
chunksize=None,
compression='infer',
thousands=None,
decimal=b'.',
lineterminator=None,
quotechar='"',
quoting=0,
escapechar=None,
comment=None,
encoding=None,
dialect=None,
tupleize_cols=None,
error_bad_lines=True,
warn_bad_lines=True,
skipfooter=0,
skip_footer=0,
doublequote=True,
delim_whitespace=False,
as_recarray=None,
compact_ints=None,
use_unsigned=None,
low_memory=True,
buffer_lines=None,
memory_map=False,
float_precision=None)
ray_frame = from_pandas(port_frame, get_npartitions())

return ray_frame
kwargs = dict(
sep=sep,
delimiter=delimiter,
header=header,
names=names,
index_col=index_col,
usecols=usecols,
squeeze=squeeze,
prefix=prefix,
mangle_dupe_cols=mangle_dupe_cols,
dtype=dtype,
engine=engine,
converters=converters,
true_values=true_values,
false_values=false_values,
skipinitialspace=skipinitialspace,
skiprows=skiprows,
nrows=nrows,
na_values=na_values,
keep_default_na=keep_default_na,
na_filter=na_filter,
verbose=verbose,
skip_blank_lines=skip_blank_lines,
parse_dates=parse_dates,
infer_datetime_format=infer_datetime_format,
keep_date_col=keep_date_col,
date_parser=date_parser,
dayfirst=dayfirst,
iterator=iterator,
chunksize=chunksize,
compression=compression,
thousands=thousands,
decimal=decimal,
lineterminator=lineterminator,
quotechar=quotechar,
quoting=quoting,
escapechar=escapechar,
comment=comment,
encoding=encoding,
dialect=dialect,
tupleize_cols=tupleize_cols,
error_bad_lines=error_bad_lines,
warn_bad_lines=warn_bad_lines,
skipfooter=skipfooter,
skip_footer=skip_footer,
doublequote=doublequote,
delim_whitespace=delim_whitespace,
as_recarray=as_recarray,
compact_ints=compact_ints,
use_unsigned=use_unsigned,
low_memory=low_memory,
buffer_lines=buffer_lines,
memory_map=memory_map,
float_precision=float_precision)

offsets = _compute_offset(filepath, get_npartitions())

first_line = _get_firstline(filepath)
columns = _infer_column(first_line, kwargs)

df_obj_ids = []
for start, end in offsets:
if start != 0:
df = _read_csv_with_offset.remote(
filepath, start, end, header=first_line, kwargs=kwargs)
else:
df = _read_csv_with_offset.remote(
filepath, start, end, kwargs=kwargs)
df_obj_ids.append(df)

return DataFrame(row_partitions=df_obj_ids, columns=columns)


def read_json(path_or_buf=None,
Expand Down