diff --git a/Cargo.lock b/Cargo.lock index d1c64e69b2..cccda23a76 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -104,8 +104,8 @@ dependencies = [ [[package]] name = "arrow2" -version = "0.17.1" -source = "git+https://github.com/Eventual-Inc/arrow2?rev=3a23c780d4d59ef0c9c8751675480e07f4e1c311#3a23c780d4d59ef0c9c8751675480e07f4e1c311" +version = "0.17.4" +source = "git+https://github.com/Eventual-Inc/arrow2?rev=d5685eebf1d65c3f3d854370ad39f93dcd91971a#d5685eebf1d65c3f3d854370ad39f93dcd91971a" dependencies = [ "ahash", "arrow-format", @@ -124,7 +124,9 @@ dependencies = [ "futures", "getrandom 0.2.10", "hash_hasher", - "hashbrown 0.13.2", + "hashbrown 0.14.1", + "indexmap 1.9.3", + "json-deserializer 0.4.4 (registry+https://github.com/rust-lang/crates.io-index)", "lexical-core", "multiversion", "num-traits", @@ -1099,10 +1101,12 @@ name = "daft" version = "0.1.10" dependencies = [ "common-daft-config", + "daft-compression", "daft-core", "daft-csv", "daft-dsl", "daft-io", + "daft-json", "daft-micropartition", "daft-parquet", "daft-plan", @@ -1115,6 +1119,15 @@ dependencies = [ "tikv-jemallocator", ] +[[package]] +name = "daft-compression" +version = "0.1.10" +dependencies = [ + "async-compression", + "tokio", + "url", +] + [[package]] name = "daft-core" version = "0.1.10" @@ -1159,6 +1172,7 @@ dependencies = [ "chrono-tz", "common-error", "csv-async", + "daft-compression", "daft-core", "daft-decoding", "daft-io", @@ -1184,11 +1198,14 @@ name = "daft-decoding" version = "0.1.10" dependencies = [ "arrow2", + "async-compression", "chrono", "chrono-tz", "csv-async", "lexical-core", "simdutf8", + "tokio", + "url", ] [[package]] @@ -1252,6 +1269,43 @@ dependencies = [ "url", ] +[[package]] +name = "daft-json" +version = "0.1.10" +dependencies = [ + "arrow2", + "async-compat", + "async-compression", + "async-stream", + "bincode", + "bytes", + "chrono", + "chrono-tz", + "common-error", + "daft-compression", + "daft-core", + "daft-decoding", + "daft-io", + "daft-table", + "futures", + "indexmap 2.1.0", + "json-deserializer 0.4.4 (git+https://github.com/Eventual-Inc/json-deserializer?rev=4be9205900888ff0767e0c37642884f2c912e22c)", + "lexical-core", + "log", + "num-traits", + "pyo3", + "pyo3-log", + "rayon", + "rstest", + "serde", + "simdutf8", + "snafu", + "tokio", + "tokio-stream", + "tokio-util", + "url", +] + [[package]] name = "daft-micropartition" version = "0.1.10" @@ -1263,6 +1317,7 @@ dependencies = [ "daft-csv", "daft-dsl", "daft-io", + "daft-json", "daft-parquet", "daft-scan", "daft-stats", @@ -1338,6 +1393,7 @@ dependencies = [ "daft-csv", "daft-dsl", "daft-io", + "daft-json", "daft-parquet", "daft-stats", "daft-table", @@ -1829,17 +1885,14 @@ version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" -[[package]] -name = "hashbrown" -version = "0.13.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" - [[package]] name = "hashbrown" version = "0.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7dfda62a12f55daeae5015f81b0baea145391cb4520f86c248fc615d72640d12" +dependencies = [ + "ahash", +] [[package]] name = "heck" @@ -2120,6 +2173,23 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "json-deserializer" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f63b421e16eb4100beb677af56f0b4f3a4f08bab74ef2af079ce5bb92c2683f" +dependencies = [ + "indexmap 1.9.3", +] + +[[package]] +name = "json-deserializer" +version = "0.4.4" +source = "git+https://github.com/Eventual-Inc/json-deserializer?rev=4be9205900888ff0767e0c37642884f2c912e22c#4be9205900888ff0767e0c37642884f2c912e22c" +dependencies = [ + "indexmap 2.1.0", +] + [[package]] name = "jsonwebtoken" version = "8.3.0" diff --git a/Cargo.toml b/Cargo.toml index ba30d0ca8a..4e8f5450f1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,9 +1,11 @@ [dependencies] common-daft-config = {path = "src/common/daft-config", default-features = false} +daft-compression = {path = "src/daft-compression", default-features = false} daft-core = {path = "src/daft-core", default-features = false} daft-csv = {path = "src/daft-csv", default-features = false} daft-dsl = {path = "src/daft-dsl", default-features = false} daft-io = {path = "src/daft-io", default-features = false} +daft-json = {path = "src/daft-json", default-features = false} daft-micropartition = {path = "src/daft-micropartition", default-features = false} daft-parquet = {path = "src/daft-parquet", default-features = false} daft-plan = {path = "src/daft-plan", default-features = false} @@ -25,6 +27,7 @@ python = [ "daft-plan/python", "daft-parquet/python", "daft-csv/python", + "daft-json/python", "daft-micropartition/python", "daft-scan/python", "daft-stats/python", @@ -76,6 +79,7 @@ members = [ "src/daft-io", "src/daft-parquet", "src/daft-csv", + "src/daft-json", "src/daft-dsl", "src/daft-table", "src/daft-plan", @@ -108,10 +112,10 @@ tokio-util = "0.7.8" url = "2.4.0" [workspace.dependencies.arrow2] -# branch = "jay/fix-parquet-timezone-parsing" +# branch = "daft-fork" git = "https://github.com/Eventual-Inc/arrow2" package = "arrow2" -rev = "3a23c780d4d59ef0c9c8751675480e07f4e1c311" +rev = "d5685eebf1d65c3f3d854370ad39f93dcd91971a" [workspace.dependencies.bincode] version = "1.3.3" diff --git a/daft/daft.pyi b/daft/daft.pyi index bb037b955a..0128dedafe 100644 --- a/daft/daft.pyi +++ b/daft/daft.pyi @@ -211,6 +211,15 @@ class JsonSourceConfig: Configuration of a JSON data source. """ + buffer_size: int | None + chunk_size: int | None + + def __init__( + self, + buffer_size: int | None = None, + chunk_size: int | None = None, + ): ... + class FileFormatConfig: """ Configuration for parsing a particular file format (Parquet, CSV, JSON). @@ -298,6 +307,41 @@ class CsvReadOptions: chunk_size: int | None = None, ): ... +class JsonConvertOptions: + """ + Options for converting JSON data to Daft data. + """ + + limit: int | None + include_columns: list[str] | None + schema: PySchema | None + + def __init__( + self, + limit: int | None = None, + include_columns: list[str] | None = None, + schema: PySchema | None = None, + ): ... + +class JsonParseOptions: + """ + Options for parsing JSON files. + """ + +class JsonReadOptions: + """ + Options for reading JSON files. + """ + + buffer_size: int | None + chunk_size: int | None + + def __init__( + self, + buffer_size: int | None = None, + chunk_size: int | None = None, + ): ... + class FileInfo: """ Metadata for a single file. @@ -587,6 +631,21 @@ def read_csv_schema( io_config: IOConfig | None = None, multithreaded_io: bool | None = None, ): ... +def read_json( + uri: str, + convert_options: JsonConvertOptions | None = None, + parse_options: JsonParseOptions | None = None, + read_options: JsonReadOptions | None = None, + io_config: IOConfig | None = None, + multithreaded_io: bool | None = None, + max_chunks_in_flight: int | None = None, +): ... +def read_json_schema( + uri: str, + parse_options: JsonParseOptions | None = None, + io_config: IOConfig | None = None, + multithreaded_io: bool | None = None, +): ... class PyTimeUnit: @staticmethod @@ -931,6 +990,16 @@ class PyMicroPartition: io_config: IOConfig | None = None, multithreaded_io: bool | None = None, ): ... + @classmethod + def read_json_native( + cls, + uri: str, + convert_options: JsonConvertOptions | None = None, + parse_options: JsonParseOptions | None = None, + read_options: JsonReadOptions | None = None, + io_config: IOConfig | None = None, + multithreaded_io: bool | None = None, + ): ... class PhysicalPlanScheduler: """ diff --git a/daft/execution/execution_step.py b/daft/execution/execution_step.py index 59c613f360..672aa0def7 100644 --- a/daft/execution/execution_step.py +++ b/daft/execution/execution_step.py @@ -17,6 +17,7 @@ FileFormatConfig, IOConfig, JoinType, + JsonReadOptions, JsonSourceConfig, ParquetSourceConfig, ResourceRequest, @@ -370,6 +371,9 @@ def _handle_tabular_files_scan( file=fp, schema=self.schema, storage_config=self.storage_config, + json_read_options=JsonReadOptions( + buffer_size=format_config.buffer_size, chunk_size=format_config.chunk_size + ), read_options=read_options, ) for fp in filepaths diff --git a/daft/io/_json.py b/daft/io/_json.py index 05d519cbe7..cb40bbea4c 100644 --- a/daft/io/_json.py +++ b/daft/io/_json.py @@ -7,6 +7,7 @@ FileFormatConfig, IOConfig, JsonSourceConfig, + NativeStorageConfig, PythonStorageConfig, StorageConfig, ) @@ -20,6 +21,9 @@ def read_json( path: Union[str, List[str]], schema_hints: Optional[Dict[str, DataType]] = None, io_config: Optional["IOConfig"] = None, + use_native_downloader: bool = True, + _buffer_size: Optional[int] = None, + _chunk_size: Optional[int] = None, ) -> DataFrame: """Creates a DataFrame from line-delimited JSON file(s) @@ -34,6 +38,8 @@ def read_json( schema_hints (dict[str, DataType]): A mapping between column names and datatypes - passing this option will disable all schema inference on data being read, and throw an error if data being read is incompatible. io_config (IOConfig): Config to be used with the native downloader + use_native_downloader: Whether to use the native downloader instead of PyArrow for reading Parquet. This + is currently experimental. returns: DataFrame: parsed DataFrame @@ -41,8 +47,11 @@ def read_json( if isinstance(path, list) and len(path) == 0: raise ValueError(f"Cannot read DataFrame from from empty list of JSON filepaths") - json_config = JsonSourceConfig() + json_config = JsonSourceConfig(_buffer_size, _chunk_size) file_format_config = FileFormatConfig.from_json_config(json_config) - storage_config = StorageConfig.python(PythonStorageConfig(io_config=io_config)) + if use_native_downloader: + storage_config = StorageConfig.native(NativeStorageConfig(True, io_config)) + else: + storage_config = StorageConfig.python(PythonStorageConfig(io_config=io_config)) builder = _get_tabular_files_scan(path, schema_hints, file_format_config, storage_config=storage_config) return DataFrame(builder) diff --git a/daft/logical/schema.py b/daft/logical/schema.py index 47c6d32569..c0a5455bd7 100644 --- a/daft/logical/schema.py +++ b/daft/logical/schema.py @@ -3,10 +3,11 @@ import sys from typing import TYPE_CHECKING, Iterator -from daft.daft import CsvParseOptions +from daft.daft import CsvParseOptions, JsonParseOptions from daft.daft import PyField as _PyField from daft.daft import PySchema as _PySchema from daft.daft import read_csv_schema as _read_csv_schema +from daft.daft import read_json_schema as _read_json_schema from daft.daft import read_parquet_schema as _read_parquet_schema from daft.datatype import DataType, TimeUnit @@ -183,3 +184,20 @@ def from_csv( multithreaded_io=multithreaded_io, ) ) + + @classmethod + def from_json( + cls, + path: str, + parse_options: JsonParseOptions | None = None, + io_config: IOConfig | None = None, + multithreaded_io: bool | None = None, + ) -> Schema: + return Schema._from_pyschema( + _read_json_schema( + uri=path, + parse_options=parse_options, + io_config=io_config, + multithreaded_io=multithreaded_io, + ) + ) diff --git a/daft/table/micropartition.py b/daft/table/micropartition.py index f931fda2bd..1ebccb51d0 100644 --- a/daft/table/micropartition.py +++ b/daft/table/micropartition.py @@ -11,6 +11,9 @@ CsvReadOptions, IOConfig, JoinType, + JsonConvertOptions, + JsonParseOptions, + JsonReadOptions, ) from daft.daft import PyMicroPartition as _PyMicroPartition from daft.daft import PyTable as _PyTable @@ -374,3 +377,24 @@ def read_csv( multithreaded_io=multithreaded_io, ) ) + + @classmethod + def read_json( + cls, + path: str, + convert_options: JsonConvertOptions | None = None, + parse_options: JsonParseOptions | None = None, + read_options: JsonReadOptions | None = None, + io_config: IOConfig | None = None, + multithreaded_io: bool | None = None, + ) -> MicroPartition: + return MicroPartition._from_pymicropartition( + _PyMicroPartition.read_json_native( + uri=path, + convert_options=convert_options, + parse_options=parse_options, + read_options=read_options, + io_config=io_config, + multithreaded_io=multithreaded_io, + ) + ) diff --git a/daft/table/schema_inference.py b/daft/table/schema_inference.py index dce4ca4f18..7be12c705c 100644 --- a/daft/table/schema_inference.py +++ b/daft/table/schema_inference.py @@ -8,6 +8,7 @@ from daft.daft import ( CsvParseOptions, + JsonParseOptions, NativeStorageConfig, PythonStorageConfig, StorageConfig, @@ -90,7 +91,16 @@ def from_json( io_config = None if storage_config is not None: config = storage_config.config - assert isinstance(config, PythonStorageConfig), "JSON schema inference only supports PythonStorageConfig" + if isinstance(config, NativeStorageConfig): + assert isinstance(file, (str, pathlib.Path)), "Native downloader only works on string inputs to read_json" + io_config = config.io_config + return Schema.from_json( + str(file), + parse_options=JsonParseOptions(), + io_config=io_config, + ) + + assert isinstance(config, PythonStorageConfig) io_config = config.io_config with _open_stream(file, io_config) as f: diff --git a/daft/table/table.py b/daft/table/table.py index 1669170e23..6b87c2ee03 100644 --- a/daft/table/table.py +++ b/daft/table/table.py @@ -6,10 +6,19 @@ import pyarrow as pa from daft.arrow_utils import ensure_table -from daft.daft import CsvConvertOptions, CsvParseOptions, CsvReadOptions, JoinType +from daft.daft import ( + CsvConvertOptions, + CsvParseOptions, + CsvReadOptions, + JoinType, + JsonConvertOptions, + JsonParseOptions, + JsonReadOptions, +) from daft.daft import PyTable as _PyTable from daft.daft import ScanTask as _ScanTask from daft.daft import read_csv as _read_csv +from daft.daft import read_json as _read_json from daft.daft import read_parquet as _read_parquet from daft.daft import read_parquet_bulk as _read_parquet_bulk from daft.daft import read_parquet_into_pyarrow as _read_parquet_into_pyarrow @@ -465,6 +474,29 @@ def read_csv( ) ) + @classmethod + def read_json( + cls, + path: str, + convert_options: JsonConvertOptions | None = None, + parse_options: JsonParseOptions | None = None, + read_options: JsonReadOptions | None = None, + io_config: IOConfig | None = None, + multithreaded_io: bool | None = None, + max_chunks_in_flight: int | None = None, + ) -> Table: + return Table._from_pytable( + _read_json( + uri=path, + convert_options=convert_options, + parse_options=parse_options, + read_options=read_options, + io_config=io_config, + multithreaded_io=multithreaded_io, + max_chunks_in_flight=max_chunks_in_flight, + ) + ) + def _trim_pyarrow_large_arrays(arr: pa.ChunkedArray) -> pa.ChunkedArray: if pa.types.is_large_binary(arr.type) or pa.types.is_large_string(arr.type): diff --git a/daft/table/table_io.py b/daft/table/table_io.py index 82f48b733c..6774ae5175 100644 --- a/daft/table/table_io.py +++ b/daft/table/table_io.py @@ -17,6 +17,9 @@ CsvParseOptions, CsvReadOptions, IOConfig, + JsonConvertOptions, + JsonParseOptions, + JsonReadOptions, NativeStorageConfig, PythonStorageConfig, StorageConfig, @@ -74,6 +77,7 @@ def read_json( file: FileInput, schema: Schema, storage_config: StorageConfig | None = None, + json_read_options: JsonReadOptions | None = None, read_options: TableReadOptions = TableReadOptions(), ) -> MicroPartition: """Reads a MicroPartition from a JSON file @@ -82,7 +86,8 @@ def read_json( file (str | IO): either a file-like object or a string file path (potentially prefixed with a protocol such as "s3://") fs (fsspec.AbstractFileSystem): fsspec FileSystem to use for reading data. By default, Daft will automatically construct a FileSystem instance internally. - read_options (TableReadOptions, optional): Options for reading the file + json_read_options (JsonReadOptions, optional): JSON-specific configs to apply when reading the file + read_options (TableReadOptions, optional): Non-format-specific options for reading the file Returns: MicroPartition: Parsed MicroPartition from JSON @@ -90,8 +95,25 @@ def read_json( io_config = None if storage_config is not None: config = storage_config.config - assert isinstance(config, PythonStorageConfig), "JSON reads only supports PyStorageConfig" - io_config = config.io_config + if isinstance(config, NativeStorageConfig): + assert isinstance(file, (str, pathlib.Path)), "Native downloader only works on string inputs to read_json" + json_convert_options = JsonConvertOptions( + limit=read_options.num_rows, + include_columns=read_options.column_names, + schema=schema._schema if schema is not None else None, + ) + json_parse_options = JsonParseOptions() + tbl = MicroPartition.read_json( + str(file), + convert_options=json_convert_options, + parse_options=json_parse_options, + read_options=json_read_options, + io_config=config.io_config, + ) + return _cast_table_to_schema(tbl, read_options=read_options, schema=schema) + else: + assert isinstance(config, PythonStorageConfig) + io_config = config.io_config with _open_stream(file, io_config) as f: table = pajson.read_json(f) @@ -130,7 +152,7 @@ def read_parquet( if isinstance(config, NativeStorageConfig): assert isinstance( file, (str, pathlib.Path) - ), "Native downloader only works on string inputs to read_parquet" + ), "Native downloader only works on string or Path inputs to read_parquet" tbl = MicroPartition.read_parquet( str(file), columns=read_options.column_names, @@ -225,7 +247,7 @@ def read_csv( if isinstance(config, NativeStorageConfig): assert isinstance( file, (str, pathlib.Path) - ), "Native downloader only works on string inputs to read_parquet" + ), "Native downloader only works on string or Path inputs to read_csv" has_header = csv_options.header_index is not None csv_convert_options = CsvConvertOptions( limit=read_options.num_rows, diff --git a/src/daft-compression/Cargo.toml b/src/daft-compression/Cargo.toml new file mode 100644 index 0000000000..6b695535e5 --- /dev/null +++ b/src/daft-compression/Cargo.toml @@ -0,0 +1,9 @@ +[dependencies] +async-compression = {workspace = true} +tokio = {workspace = true} +url = {workspace = true} + +[package] +edition = {workspace = true} +name = "daft-compression" +version = {workspace = true} diff --git a/src/daft-compression/src/compression.rs b/src/daft-compression/src/compression.rs new file mode 100644 index 0000000000..268b1566d9 --- /dev/null +++ b/src/daft-compression/src/compression.rs @@ -0,0 +1,66 @@ +use async_compression::tokio::bufread::{ + BrotliDecoder, BzDecoder, DeflateDecoder, GzipDecoder, LzmaDecoder, XzDecoder, ZlibDecoder, + ZstdDecoder, +}; +use std::{path::PathBuf, pin::Pin}; +use tokio::io::{AsyncBufRead, AsyncRead}; +use url::Url; + +#[derive(Debug)] +pub enum CompressionCodec { + Brotli, + Bz, + Deflate, + Gzip, + Lzma, + Xz, + Zlib, + Zstd, +} + +impl CompressionCodec { + pub fn from_uri(uri: &str) -> Option { + let url = Url::parse(uri); + let path = match &url { + Ok(url) => url.path(), + _ => uri, + }; + let extension = PathBuf::from(path) + .extension()? + .to_string_lossy() + .to_string(); + Self::from_extension(extension.as_ref()) + } + pub fn from_extension(extension: &str) -> Option { + use CompressionCodec::*; + match extension { + "br" => Some(Brotli), + "bz2" => Some(Bz), + "deflate" => Some(Deflate), + "gz" => Some(Gzip), + "lzma" => Some(Lzma), + "xz" => Some(Xz), + "zl" => Some(Zlib), + "zstd" | "zst" => Some(Zstd), + "snappy" => todo!("Snappy compression support not yet implemented"), + _ => None, + } + } + + pub fn to_decoder( + &self, + reader: T, + ) -> Pin> { + use CompressionCodec::*; + match self { + Brotli => Box::pin(BrotliDecoder::new(reader)), + Bz => Box::pin(BzDecoder::new(reader)), + Deflate => Box::pin(DeflateDecoder::new(reader)), + Gzip => Box::pin(GzipDecoder::new(reader)), + Lzma => Box::pin(LzmaDecoder::new(reader)), + Xz => Box::pin(XzDecoder::new(reader)), + Zlib => Box::pin(ZlibDecoder::new(reader)), + Zstd => Box::pin(ZstdDecoder::new(reader)), + } + } +} diff --git a/src/daft-compression/src/lib.rs b/src/daft-compression/src/lib.rs new file mode 100644 index 0000000000..9b869b3345 --- /dev/null +++ b/src/daft-compression/src/lib.rs @@ -0,0 +1,4 @@ +//! Utilities for async decompression of data. +pub mod compression; + +pub use compression::CompressionCodec; diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index d0e164b7d9..620316e8d0 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -9,6 +9,7 @@ chrono = {workspace = true} chrono-tz = {workspace = true} common-error = {path = "../common/error", default-features = false} csv-async = "1.2.6" +daft-compression = {path = "../daft-compression", default-features = false} daft-core = {path = "../daft-core", default-features = false} daft-decoding = {path = "../daft-decoding"} daft-io = {path = "../daft-io", default-features = false} diff --git a/src/daft-csv/src/lib.rs b/src/daft-csv/src/lib.rs index 7541563578..10c3d024a6 100644 --- a/src/daft-csv/src/lib.rs +++ b/src/daft-csv/src/lib.rs @@ -5,7 +5,6 @@ use common_error::DaftError; use snafu::Snafu; -mod compression; pub mod metadata; pub mod options; #[cfg(feature = "python")] diff --git a/src/daft-csv/src/metadata.rs b/src/daft-csv/src/metadata.rs index 75ac0ad43a..f45f25946b 100644 --- a/src/daft-csv/src/metadata.rs +++ b/src/daft-csv/src/metadata.rs @@ -14,7 +14,8 @@ use tokio::{ }; use tokio_util::io::StreamReader; -use crate::{compression::CompressionCodec, schema::merge_schema, CsvParseOptions}; +use crate::{schema::merge_schema, CsvParseOptions}; +use daft_compression::CompressionCodec; use daft_decoding::inference::infer; const DEFAULT_COLUMN_PREFIX: &str = "column_"; diff --git a/src/daft-csv/src/read.rs b/src/daft-csv/src/read.rs index 460e01f377..772bbca00b 100644 --- a/src/daft-csv/src/read.rs +++ b/src/daft-csv/src/read.rs @@ -25,8 +25,9 @@ use tokio::{ }; use tokio_util::io::StreamReader; -use crate::{compression::CompressionCodec, ArrowSnafu}; +use crate::ArrowSnafu; use crate::{metadata::read_csv_schema_single, CsvConvertOptions, CsvParseOptions, CsvReadOptions}; +use daft_compression::CompressionCodec; use daft_decoding::deserialize::deserialize_column; trait ByteRecordChunkStream = Stream>>; @@ -81,64 +82,58 @@ pub fn read_csv_bulk( ) -> DaftResult> { let runtime_handle = get_runtime(multithreaded_io)?; let _rt_guard = runtime_handle.enter(); - let tables = runtime_handle - .block_on(async move { - // Launch a read task per URI, throttling the number of concurrent file reads to num_parallel tasks. - let task_stream = futures::stream::iter(uris.iter().enumerate().map(|(i, uri)| { - let (uri, convert_options, parse_options, read_options, io_client, io_stats) = ( - uri.to_string(), - convert_options.clone(), - parse_options.clone(), - read_options.clone(), - io_client.clone(), - io_stats.clone(), - ); - tokio::task::spawn(async move { - let table = read_csv_single_into_table( - uri.as_str(), - convert_options, - parse_options, - read_options, - io_client, - io_stats, - max_chunks_in_flight, - ) - .await?; - Ok((i, table)) - }) - })); - let mut remaining_rows = convert_options - .as_ref() - .and_then(|opts| opts.limit.map(|limit| limit as i64)); - task_stream - // Each task is annotated with its position in the output, so we can use unordered buffering to help mitigate stragglers - // and sort the task results at the end. - .buffer_unordered(num_parallel_tasks) - // Terminate the stream if we have already reached the row limit. With the upstream buffering, we will still read up to - // num_parallel_tasks redundant files. - .try_take_while(|result| { - match (result, remaining_rows) { - // Limit has been met, early-teriminate. - (_, Some(rows_left)) if rows_left <= 0 => futures::future::ready(Ok(false)), - // Limit has not yet been met, update remaining limit slack and continue. - (Ok((_, table)), Some(rows_left)) => { - remaining_rows = Some(rows_left - table.len() as i64); - futures::future::ready(Ok(true)) - } - // (1) No limit, never early-terminate. - // (2) Encountered error, propagate error to try_collect to allow it to short-circuit. - (_, None) | (Err(_), _) => futures::future::ready(Ok(true)), - } - }) - .try_collect::>() + let tables = runtime_handle.block_on(async move { + // Launch a read task per URI, throttling the number of concurrent file reads to num_parallel tasks. + let task_stream = futures::stream::iter(uris.iter().map(|uri| { + let (uri, convert_options, parse_options, read_options, io_client, io_stats) = ( + uri.to_string(), + convert_options.clone(), + parse_options.clone(), + read_options.clone(), + io_client.clone(), + io_stats.clone(), + ); + tokio::task::spawn(async move { + read_csv_single_into_table( + uri.as_str(), + convert_options, + parse_options, + read_options, + io_client, + io_stats, + max_chunks_in_flight, + ) .await - }) - .context(super::JoinSnafu {})?; + }) + .context(super::JoinSnafu {}) + })); + let mut remaining_rows = convert_options + .as_ref() + .and_then(|opts| opts.limit.map(|limit| limit as i64)); + task_stream + // Limit the number of file reads we have in flight at any given time. + .buffered(num_parallel_tasks) + // Terminate the stream if we have already reached the row limit. With the upstream buffering, we will still read up to + // num_parallel_tasks redundant files. + .try_take_while(|result| { + match (result, remaining_rows) { + // Limit has been met, early-teriminate. + (_, Some(rows_left)) if rows_left <= 0 => futures::future::ready(Ok(false)), + // Limit has not yet been met, update remaining limit slack and continue. + (Ok(table), Some(rows_left)) => { + remaining_rows = Some(rows_left - table.len() as i64); + futures::future::ready(Ok(true)) + } + // (1) No limit, never early-terminate. + // (2) Encountered error, propagate error to try_collect to allow it to short-circuit. + (_, None) | (Err(_), _) => futures::future::ready(Ok(true)), + } + }) + .try_collect::>() + .await + })?; - // Sort the task results by task index, yielding tables whose order matches the input URI order. - let mut collected = tables.into_iter().collect::>>()?; - collected.sort_by_key(|(idx, _)| *idx); - Ok(collected.into_iter().map(|(_, v)| v).collect()) + tables.into_iter().collect::>>() } async fn read_csv_single_into_table( @@ -1175,11 +1170,11 @@ mod tests { file.as_ref(), None, None, - None, + Some(CsvReadOptions::default().with_chunk_size(Some(5))), io_client, None, true, - Some(5), + Some(2), )?; assert_eq!(table.len(), 20); assert_eq!( diff --git a/src/daft-decoding/Cargo.toml b/src/daft-decoding/Cargo.toml index 4a054d0df9..e0eb888cd6 100644 --- a/src/daft-decoding/Cargo.toml +++ b/src/daft-decoding/Cargo.toml @@ -1,10 +1,13 @@ [dependencies] arrow2 = {workspace = true, features = ["io_csv", "io_csv_async"]} +async-compression = {workspace = true} chrono = {workspace = true} chrono-tz = {workspace = true} csv-async = "1.2.6" lexical-core = {version = "0.8"} simdutf8 = "0.1.3" +tokio = {workspace = true} +url = {workspace = true} [package] edition = {workspace = true} diff --git a/src/daft-decoding/src/deserialize.rs b/src/daft-decoding/src/deserialize.rs index e8e79156a6..a57a2c2ac2 100644 --- a/src/daft-decoding/src/deserialize.rs +++ b/src/daft-decoding/src/deserialize.rs @@ -12,6 +12,8 @@ use csv_async::ByteRecord; pub(crate) const ISO8601: &str = "%+"; pub(crate) const ISO8601_NO_TIME_ZONE: &str = "%Y-%m-%dT%H:%M:%S%.f"; pub(crate) const ISO8601_NO_TIME_ZONE_NO_FRACTIONAL: &str = "%Y-%m-%dT%H:%M:%S"; +pub(crate) const ISO8601_DATE: &str = "%Y-%m-%d"; +pub(crate) const ISO8601_DATE_SLASHES: &str = "%Y/%m/%d"; pub(crate) const RFC3339_WITH_SPACE: &str = "%Y-%m-%d %H:%M:%S%.f%:z"; pub(crate) const RFC3339_WITH_SPACE_NO_TIME_ZONE: &str = "%Y-%m-%d %H:%M:%S%.f"; pub(crate) const RFC3339_WITH_SPACE_NO_TIME_ZONE_NO_FRACTIONAL: &str = "%Y-%m-%d %H:%M:%S"; @@ -20,8 +22,11 @@ pub(crate) const ALL_NAIVE_TIMESTAMP_FMTS: &[&str] = &[ ISO8601_NO_TIME_ZONE_NO_FRACTIONAL, RFC3339_WITH_SPACE_NO_TIME_ZONE, RFC3339_WITH_SPACE_NO_TIME_ZONE_NO_FRACTIONAL, + ISO8601_DATE, + ISO8601_DATE_SLASHES, ]; pub(crate) const ALL_TIMESTAMP_FMTS: &[&str] = &[ISO8601, RFC3339_WITH_SPACE]; +pub(crate) const ALL_NAIVE_DATE_FMTS: &[&str] = &[ISO8601_DATE, ISO8601_DATE_SLASHES]; // Ideally this trait should not be needed and both `csv` and `csv_async` crates would share // the same `ByteRecord` struct. Unfortunately, they do not and thus we must use generics @@ -153,7 +158,24 @@ fn deserialize_null(rows: &[B], _: usize) -> Box Option { +pub fn deserialize_naive_date(string: &str, fmt_idx: &mut usize) -> Option { + // TODO(Clark): Parse as all candidate formats in a single pass. + for i in 0..ALL_NAIVE_DATE_FMTS.len() { + let idx = (i + *fmt_idx) % ALL_NAIVE_DATE_FMTS.len(); + let fmt = ALL_NAIVE_DATE_FMTS[idx]; + if let Ok(dt) = chrono::NaiveDate::parse_from_str(string, fmt) { + *fmt_idx = idx; + return Some(dt); + } + } + None +} + +#[inline] +pub fn deserialize_naive_datetime( + string: &str, + fmt_idx: &mut usize, +) -> Option { // TODO(Clark): Parse as all candidate formats in a single pass. for i in 0..ALL_NAIVE_TIMESTAMP_FMTS.len() { let idx = (i + *fmt_idx) % ALL_NAIVE_TIMESTAMP_FMTS.len(); @@ -167,7 +189,7 @@ fn deserialize_naive_datetime(string: &str, fmt_idx: &mut usize) -> Option( +pub fn deserialize_datetime( string: &str, tz: &T, fmt_idx: &mut usize, @@ -234,13 +256,15 @@ pub fn deserialize_column( lexical_core::parse::(bytes).ok() }), Date32 => deserialize_primitive(rows, column, datatype, |bytes| { + let mut last_fmt_idx = 0; to_utf8(bytes) - .and_then(|x| x.parse::().ok()) + .and_then(|x| deserialize_naive_date(x, &mut last_fmt_idx)) .map(|x| x.num_days_from_ce() - temporal_conversions::EPOCH_DAYS_FROM_CE) }), Date64 => deserialize_primitive(rows, column, datatype, |bytes| { + let mut last_fmt_idx = 0; to_utf8(bytes) - .and_then(|x| x.parse::().ok()) + .and_then(|x| deserialize_naive_datetime(x, &mut last_fmt_idx)) .map(|x| x.timestamp_millis()) }), Time32(time_unit) => deserialize_primitive(rows, column, datatype, |bytes| { @@ -310,7 +334,7 @@ pub fn deserialize_column( } // Return the factor by how small is a time unit compared to seconds -fn get_factor_from_timeunit(time_unit: TimeUnit) -> u32 { +pub fn get_factor_from_timeunit(time_unit: TimeUnit) -> u32 { match time_unit { TimeUnit::Second => 1, TimeUnit::Millisecond => 1_000, diff --git a/src/daft-decoding/src/inference.rs b/src/daft-decoding/src/inference.rs index e1f4fb259f..b91630c9a0 100644 --- a/src/daft-decoding/src/inference.rs +++ b/src/daft-decoding/src/inference.rs @@ -1,7 +1,7 @@ -use arrow2::datatypes::TimeUnit; +use arrow2::datatypes::{DataType, TimeUnit}; use chrono::Timelike; -use crate::deserialize::{ALL_NAIVE_TIMESTAMP_FMTS, ALL_TIMESTAMP_FMTS}; +use crate::deserialize::{ALL_NAIVE_DATE_FMTS, ALL_NAIVE_TIMESTAMP_FMTS, ALL_TIMESTAMP_FMTS}; /// Infers [`DataType`] from `bytes` /// # Implementation @@ -16,7 +16,6 @@ use crate::deserialize::{ALL_NAIVE_TIMESTAMP_FMTS, ALL_TIMESTAMP_FMTS}; /// * other utf8 is mapped to [`DataType::Utf8`] /// * invalid utf8 is mapped to [`DataType::Binary`] pub fn infer(bytes: &[u8]) -> arrow2::datatypes::DataType { - use arrow2::datatypes::DataType; if is_null(bytes) { DataType::Null } else if is_boolean(bytes) { @@ -26,23 +25,30 @@ pub fn infer(bytes: &[u8]) -> arrow2::datatypes::DataType { } else if is_float(bytes) { DataType::Float64 } else if let Ok(string) = simdutf8::basic::from_utf8(bytes) { - if is_date(string) { - DataType::Date32 - } else if is_time(string) { - DataType::Time32(TimeUnit::Millisecond) - } else if let Some(time_unit) = is_naive_datetime(string) { - DataType::Timestamp(time_unit, None) - } else if let Some((time_unit, offset)) = is_datetime(string) { - DataType::Timestamp(time_unit, Some(offset)) - } else { - DataType::Utf8 - } + infer_string(string) } else { // invalid utf8 DataType::Binary } } +pub fn infer_string(string: &str) -> DataType { + if is_date(string) { + DataType::Date32 + } else if let Some(time_unit) = is_time(string) { + DataType::Time32(time_unit) + } else if let Some((time_unit, offset)) = is_datetime(string) { + // NOTE: We try to parse as a non-naive datatime (with timezone information) first, + // since is_datetime() will return false if timezone information is not present in the string, + // while is_naive_datetime() will ignore timezone information in the string. + DataType::Timestamp(time_unit, Some(offset)) + } else if let Some(time_unit) = is_naive_datetime(string) { + DataType::Timestamp(time_unit, None) + } else { + DataType::Utf8 + } +} + fn is_null(bytes: &[u8]) -> bool { bytes.is_empty() } @@ -60,11 +66,20 @@ fn is_integer(bytes: &[u8]) -> bool { } fn is_date(string: &str) -> bool { - string.parse::().is_ok() + for fmt in ALL_NAIVE_DATE_FMTS { + if chrono::NaiveDate::parse_from_str(string, fmt).is_ok() { + return true; + } + } + false } -fn is_time(string: &str) -> bool { - string.parse::().is_ok() +fn is_time(string: &str) -> Option { + if let Ok(t) = string.parse::() { + let time_unit = nanoseconds_to_time_unit(t.nanosecond()); + return Some(time_unit); + } + None } fn is_naive_datetime(string: &str) -> Option { diff --git a/src/daft-json/Cargo.toml b/src/daft-json/Cargo.toml new file mode 100644 index 0000000000..74060528e0 --- /dev/null +++ b/src/daft-json/Cargo.toml @@ -0,0 +1,48 @@ +[dependencies] +arrow2 = {workspace = true, features = ["io_json"]} +async-compat = {workspace = true} +async-compression = {workspace = true} +async-stream = {workspace = true} +bincode = {workspace = true} +bytes = {workspace = true} +chrono = {workspace = true} +chrono-tz = {workspace = true} +common-error = {path = "../common/error", default-features = false} +daft-compression = {path = "../daft-compression", default-features = false} +daft-core = {path = "../daft-core", default-features = false} +daft-decoding = {path = "../daft-decoding"} +daft-io = {path = "../daft-io", default-features = false} +daft-table = {path = "../daft-table", default-features = false} +futures = {workspace = true} +indexmap = {workspace = true} +lexical-core = {version = "0.8"} +log = {workspace = true} +num-traits = {workspace = true} +pyo3 = {workspace = true, optional = true} +pyo3-log = {workspace = true, optional = true} +rayon = {workspace = true} +serde = {workspace = true} +simdutf8 = "0.1.3" +snafu = {workspace = true} +tokio = {workspace = true} +tokio-stream = {workspace = true, features = ["io-util"]} +tokio-util = {workspace = true} +url = {workspace = true} + +[dependencies.json-deserializer] +features = ["preserve_order"] +git = "https://github.com/Eventual-Inc/json-deserializer" +package = "json-deserializer" +rev = "4be9205900888ff0767e0c37642884f2c912e22c" + +[dev-dependencies] +rstest = {workspace = true} + +[features] +default = ["python"] +python = ["dep:pyo3", "dep:pyo3-log", "common-error/python", "daft-core/python", "daft-io/python", "daft-table/python"] + +[package] +edition = {workspace = true} +name = "daft-json" +version = {workspace = true} diff --git a/src/daft-json/src/decoding.rs b/src/daft-json/src/decoding.rs new file mode 100644 index 0000000000..e8de99b8e3 --- /dev/null +++ b/src/daft-json/src/decoding.rs @@ -0,0 +1,537 @@ +use std::borrow::Borrow; + +use arrow2::array::{ + Array, MutableArray, MutableBooleanArray, MutableFixedSizeListArray, MutableListArray, + MutableNullArray, MutablePrimitiveArray, MutableStructArray, MutableUtf8Array, +}; +use arrow2::bitmap::MutableBitmap; +use arrow2::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; +use arrow2::error::{Error, Result}; +use arrow2::offset::Offsets; +use arrow2::temporal_conversions; +use arrow2::types::{f16, NativeType, Offset}; +use chrono::{Datelike, Timelike}; +use daft_decoding::deserialize::{ + deserialize_datetime, deserialize_naive_date, deserialize_naive_datetime, + get_factor_from_timeunit, +}; +use indexmap::IndexMap; +use json_deserializer::{Number, Value}; + +/// Deserialize chunk of JSON records into a chunk of Arrow2 arrays. +pub(crate) fn deserialize_records( + records: Vec, + schema: &Schema, + schema_is_projection: bool, +) -> Result>> { + // Allocate mutable arrays. + let mut results = schema + .fields + .iter() + .map(|f| (&f.name, allocate_array(f, records.len()))) + .collect::>(); + + for record in records { + match record { + Value::Object(record) => { + for (key, value) in record.iter() { + let arr = results.get_mut(key); + if let Some(arr) = arr { + deserialize_into(arr, &[value]); + } else if !schema_is_projection { + // Provided schema is either the full schema or a projection. + // If this key isn't in the schema-derived array map AND there was no projection, + // we return an error. Otherwise, we drop this key-value pair. + return Err(Error::ExternalFormat(format!("unexpected key: '{key}'"))); + } + } + } + _ => { + return Err(Error::ExternalFormat(format!( + "Each line in a newline-delimited JSON file must be a JSON object, but got: {:?}", + record + ))) + } + } + } + + Ok(results.into_values().map(|mut ma| ma.as_box()).collect()) +} + +fn allocate_array(f: &Field, length: usize) -> Box { + match f.data_type() { + DataType::Null => Box::new(MutableNullArray::new(DataType::Null, 0)), + DataType::Int8 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::Int16 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + dt @ (DataType::Int32 + | DataType::Date32 + | DataType::Time32(_) + | DataType::Interval(IntervalUnit::YearMonth)) => { + Box::new(MutablePrimitiveArray::::with_capacity(length).to(dt.clone())) + } + dt @ (DataType::Int64 + | DataType::Date64 + | DataType::Time64(_) + | DataType::Duration(_) + | DataType::Timestamp(..)) => { + Box::new(MutablePrimitiveArray::::with_capacity(length).to(dt.clone())) + } + DataType::UInt8 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::UInt16 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::UInt32 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::UInt64 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::Float16 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::Float32 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::Float64 => Box::new(MutablePrimitiveArray::::with_capacity(length)), + DataType::Boolean => Box::new(MutableBooleanArray::with_capacity(length)), + DataType::Utf8 => Box::new(MutableUtf8Array::::with_capacity(length)), + DataType::LargeUtf8 => Box::new(MutableUtf8Array::::with_capacity(length)), + DataType::FixedSizeList(inner, size) => Box::new(MutableFixedSizeListArray::new_from( + allocate_array(inner, length), + f.data_type().clone(), + *size, + )), + // TODO(Clark): Ensure that these mutable list arrays work correctly and efficiently for arbitrarily nested arrays. + // TODO(Clark): We have to manually give a non-None bitmap due to a bug in try_extend_from_lengths for + // mutable list arrays, which will unintentionally drop the validity mask if the bitmap isn't already non-None. + DataType::List(inner) => Box::new(MutableListArray::new_from_mutable( + allocate_array(inner, length), + Offsets::::with_capacity(length), + Some(MutableBitmap::with_capacity(length)), + )), + DataType::LargeList(inner) => Box::new(MutableListArray::new_from_mutable( + allocate_array(inner, length), + Offsets::::with_capacity(length), + Some(MutableBitmap::with_capacity(length)), + )), + // TODO(Clark): We have to manually give a non-None bitmap due to a bug in MutableStructArray::push(), which will + // unintentionally drop the first null added to the validity mask if a bitmap hasn't been initialized from the start. + dt @ DataType::Struct(inner) => Box::new( + MutableStructArray::try_new( + dt.clone(), + inner + .iter() + .map(|field| allocate_array(field, length)) + .collect::>(), + Some(MutableBitmap::with_capacity(length)), + ) + .unwrap(), + ), + dt => todo!("Dtype not supported: {:?}", dt), + } +} + +/// Deserialize `rows` by extending them into the given `target` +fn deserialize_into<'a, A: Borrow>>(target: &mut Box, rows: &[A]) { + match target.data_type() { + DataType::Null => { + // TODO(Clark): Return an error if any of rows are not Value::Null. + for _ in 0..rows.len() { + target.push_null() + } + } + DataType::Boolean => generic_deserialize_into(target, rows, deserialize_boolean_into), + DataType::Float32 => { + deserialize_primitive_into::<_, f32>(target, rows, deserialize_float_into) + } + DataType::Float64 => { + deserialize_primitive_into::<_, f64>(target, rows, deserialize_float_into) + } + DataType::Int8 => deserialize_primitive_into::<_, i8>(target, rows, deserialize_int_into), + DataType::Int16 => deserialize_primitive_into::<_, i16>(target, rows, deserialize_int_into), + DataType::Int32 | DataType::Interval(IntervalUnit::YearMonth) => { + deserialize_primitive_into::<_, i32>(target, rows, deserialize_int_into) + } + DataType::Date32 | DataType::Time32(_) => { + deserialize_primitive_into::<_, i32>(target, rows, deserialize_date_into) + } + DataType::Interval(IntervalUnit::DayTime) => { + unimplemented!("There is no natural representation of DayTime in JSON.") + } + DataType::Int64 | DataType::Duration(_) => { + deserialize_primitive_into::<_, i64>(target, rows, deserialize_int_into) + } + DataType::Timestamp(..) | DataType::Date64 | DataType::Time64(_) => { + deserialize_primitive_into::<_, i64>(target, rows, deserialize_datetime_into) + } + DataType::UInt8 => deserialize_primitive_into::<_, u8>(target, rows, deserialize_int_into), + DataType::UInt16 => { + deserialize_primitive_into::<_, u16>(target, rows, deserialize_int_into) + } + DataType::UInt32 => { + deserialize_primitive_into::<_, u32>(target, rows, deserialize_int_into) + } + DataType::UInt64 => { + deserialize_primitive_into::<_, u64>(target, rows, deserialize_int_into) + } + DataType::Utf8 => generic_deserialize_into::<_, MutableUtf8Array>( + target, + rows, + deserialize_utf8_into, + ), + DataType::LargeUtf8 => generic_deserialize_into::<_, MutableUtf8Array>( + target, + rows, + deserialize_utf8_into, + ), + DataType::FixedSizeList(_, _) => { + generic_deserialize_into(target, rows, deserialize_fixed_size_list_into) + } + DataType::List(_) => deserialize_list_into( + target + .as_mut_any() + .downcast_mut::>>() + .unwrap(), + rows, + ), + DataType::LargeList(_) => deserialize_list_into( + target + .as_mut_any() + .downcast_mut::>>() + .unwrap(), + rows, + ), + DataType::Struct(_) => { + generic_deserialize_into::<_, MutableStructArray>(target, rows, deserialize_struct_into) + } + // TODO(Clark): Add support for decimal type. + // TODO(Clark): Add support for binary and large binary types. + dt => { + todo!("Dtype not supported: {:?}", dt) + } + } +} + +fn deserialize_primitive_into<'a, A: Borrow>, T: NativeType>( + target: &mut Box, + rows: &[A], + deserialize_into: fn(&mut MutablePrimitiveArray, &[A]) -> (), +) { + generic_deserialize_into(target, rows, deserialize_into) +} + +fn generic_deserialize_into<'a, A: Borrow>, M: 'static>( + target: &mut Box, + rows: &[A], + deserialize_into: fn(&mut M, &[A]) -> (), +) { + deserialize_into(target.as_mut_any().downcast_mut::().unwrap(), rows); +} + +fn deserialize_utf8_into<'a, O: Offset, A: Borrow>>( + target: &mut MutableUtf8Array, + rows: &[A], +) { + let mut scratch = vec![]; + for row in rows { + match row.borrow() { + Value::String(v) => target.push(Some(v.as_ref())), + Value::Number(number) => match number { + Number::Integer(number, exponent) | Number::Float(number, exponent) => { + scratch.clear(); + scratch.extend_from_slice(number); + if !exponent.is_empty() { + scratch.push(b'e'); + scratch.extend_from_slice(exponent); + } + target.push(simdutf8::basic::from_utf8(scratch.as_slice()).ok()); + } + }, + Value::Bool(v) => target.push(Some(if *v { "true" } else { "false" })), + _ => target.push_null(), + } + } +} + +fn deserialize_int_into< + 'a, + T: NativeType + lexical_core::FromLexical + Pow10, + A: Borrow>, +>( + target: &mut MutablePrimitiveArray, + rows: &[A], +) { + let iter = rows.iter().map(|row| match row.borrow() { + Value::Number(number) => Some(deserialize_int_single(*number)), + Value::Bool(number) => Some(if *number { T::one() } else { T::default() }), + _ => None, + }); + target.extend_trusted_len(iter); +} + +fn deserialize_float_into< + 'a, + T: NativeType + lexical_core::FromLexical + Powi10, + A: Borrow>, +>( + target: &mut MutablePrimitiveArray, + rows: &[A], +) { + let iter = rows.iter().map(|row| match row.borrow() { + Value::Number(number) => Some(deserialize_float_single(number)), + Value::Bool(number) => Some(if *number { T::one() } else { T::default() }), + _ => None, + }); + target.extend_trusted_len(iter); +} + +fn deserialize_boolean_into<'a, A: Borrow>>( + target: &mut MutableBooleanArray, + rows: &[A], +) { + let iter = rows.iter().map(|row| match row.borrow() { + Value::Bool(v) => Some(v), + _ => None, + }); + target.extend_trusted_len(iter); +} + +fn deserialize_int_single(number: Number) -> T +where + T: NativeType + lexical_core::FromLexical + Pow10, +{ + match number { + Number::Float(fraction, exponent) => { + let integer = fraction.split(|x| *x == b'.').next().unwrap(); + let mut integer: T = lexical_core::parse(integer).unwrap(); + if !exponent.is_empty() { + let exponent: u32 = lexical_core::parse(exponent).unwrap(); + integer = integer.pow10(exponent); + } + integer + } + Number::Integer(integer, exponent) => { + let mut integer: T = lexical_core::parse(integer).unwrap(); + if !exponent.is_empty() { + let exponent: u32 = lexical_core::parse(exponent).unwrap(); + integer = integer.pow10(exponent); + } + integer + } + } +} + +trait Powi10: NativeType + num_traits::One + std::ops::Add { + fn powi10(self, exp: i32) -> Self; +} + +impl Powi10 for f32 { + #[inline] + fn powi10(self, exp: i32) -> Self { + self * 10.0f32.powi(exp) + } +} + +impl Powi10 for f64 { + #[inline] + fn powi10(self, exp: i32) -> Self { + self * 10.0f64.powi(exp) + } +} + +trait Pow10: NativeType + num_traits::One + std::ops::Add { + fn pow10(self, exp: u32) -> Self; +} + +macro_rules! impl_pow10 { + ($ty:ty) => { + impl Pow10 for $ty { + #[inline] + fn pow10(self, exp: u32) -> Self { + self * (10 as $ty).pow(exp) + } + } + }; +} +impl_pow10!(u8); +impl_pow10!(u16); +impl_pow10!(u32); +impl_pow10!(u64); +impl_pow10!(i8); +impl_pow10!(i16); +impl_pow10!(i32); +impl_pow10!(i64); + +fn deserialize_float_single(number: &Number) -> T +where + T: NativeType + lexical_core::FromLexical + Powi10, +{ + match number { + Number::Float(float, exponent) => { + let mut float: T = lexical_core::parse(float).unwrap(); + if !exponent.is_empty() { + let exponent: i32 = lexical_core::parse(exponent).unwrap(); + float = float.powi10(exponent); + } + float + } + Number::Integer(integer, exponent) => { + let mut float: T = lexical_core::parse(integer).unwrap(); + if !exponent.is_empty() { + let exponent: i32 = lexical_core::parse(exponent).unwrap(); + float = float.powi10(exponent); + } + float + } + } +} + +fn deserialize_date_into<'a, A: Borrow>>( + target: &mut MutablePrimitiveArray, + rows: &[A], +) { + let dtype = target.data_type().clone(); + let mut last_fmt_idx = 0; + let iter = rows.iter().map(|row| match row.borrow() { + Value::Number(v) => Some(deserialize_int_single(*v)), + Value::String(v) => match dtype { + DataType::Time32(tu) => { + let factor = get_factor_from_timeunit(tu); + v.parse::().ok().map(|x| { + (x.hour() * 3_600 * factor + + x.minute() * 60 * factor + + x.second() * factor + + x.nanosecond() / (1_000_000_000 / factor)) as i32 + }) + } + DataType::Date32 => deserialize_naive_date(v, &mut last_fmt_idx) + .map(|x| x.num_days_from_ce() - temporal_conversions::EPOCH_DAYS_FROM_CE), + _ => unreachable!(), + }, + _ => None, + }); + target.extend_trusted_len(iter); +} + +fn deserialize_datetime_into<'a, A: Borrow>>( + target: &mut MutablePrimitiveArray, + rows: &[A], +) { + let dtype = target.data_type().clone(); + let mut last_fmt_idx = 0; + let iter = rows.iter().map(|row| match row.borrow() { + Value::Number(v) => Some(deserialize_int_single(*v)), + Value::String(v) => match dtype { + DataType::Time64(tu) => { + let factor = get_factor_from_timeunit(tu) as u64; + v.parse::().ok().map(|x| { + (x.hour() as u64 * 3_600 * factor + + x.minute() as u64 * 60 * factor + + x.second() as u64 * factor + + x.nanosecond() as u64 / (1_000_000_000 / factor)) + as i64 + }) + } + DataType::Date64 => { + deserialize_naive_datetime(v, &mut last_fmt_idx).map(|x| x.timestamp_millis()) + } + DataType::Timestamp(tu, None) => deserialize_naive_datetime(v, &mut last_fmt_idx) + .and_then(|dt| match tu { + TimeUnit::Second => Some(dt.timestamp()), + TimeUnit::Millisecond => Some(dt.timestamp_millis()), + TimeUnit::Microsecond => Some(dt.timestamp_micros()), + TimeUnit::Nanosecond => dt.timestamp_nanos_opt(), + }), + DataType::Timestamp(tu, Some(ref tz)) => { + let tz = if tz == "Z" { "UTC" } else { tz }; + let tz = temporal_conversions::parse_offset(tz).unwrap(); + deserialize_datetime(v, &tz, &mut last_fmt_idx).and_then(|dt| match tu { + TimeUnit::Second => Some(dt.timestamp()), + TimeUnit::Millisecond => Some(dt.timestamp_millis()), + TimeUnit::Microsecond => Some(dt.timestamp_micros()), + TimeUnit::Nanosecond => dt.timestamp_nanos_opt(), + }) + } + _ => unreachable!(), + }, + _ => None, + }); + target.extend_trusted_len(iter); +} + +fn deserialize_list_into<'a, O: Offset, A: Borrow>>( + target: &mut MutableListArray>, + rows: &[A], +) { + let empty = vec![]; + let inner: Vec<_> = rows + .iter() + .flat_map(|row| match row.borrow() { + Value::Array(value) => value.iter(), + _ => empty.iter(), + }) + .collect(); + + deserialize_into(target.mut_values(), &inner); + + let lengths = rows.iter().map(|row| match row.borrow() { + Value::Array(value) => Some(value.len()), + _ => None, + }); + + // NOTE(Clark): A bug in Arrow2 will cause the validity mask to be dropped if it's currently None in target, + // which will be the case unless we explicitly initialize the mutable array with a bitmap. + target + .try_extend_from_lengths(lengths) + .expect("Offsets overflow"); +} + +fn deserialize_fixed_size_list_into<'a, A: Borrow>>( + target: &mut MutableFixedSizeListArray>, + rows: &[A], +) { + for row in rows { + match row.borrow() { + Value::Array(value) => { + if value.len() == target.size() { + deserialize_into(target.mut_values(), value); + // Unless alignment is already off, the if above should + // prevent this from ever happening. + target.try_push_valid().expect("unaligned backing array"); + } else { + // TODO(Clark): Return an error instead of dropping incorrectly sized lists. + target.push_null(); + } + } + _ => target.push_null(), + } + } +} + +fn deserialize_struct_into<'a, A: Borrow>>(target: &mut MutableStructArray, rows: &[A]) { + let dtype = target.data_type().clone(); + // Build a map from struct field -> JSON values. + let mut values = match dtype { + DataType::Struct(fields) => fields + .into_iter() + .map(|field| (field.name, vec![])) + .collect::>(), + _ => unreachable!(), + }; + rows.iter().for_each(|row| { + match row.borrow() { + Value::Object(value) => { + values.iter_mut().for_each(|(s, inner)| { + inner.push(value.get(s).unwrap_or(&Value::Null)); + }); + target.push(true); + } + _ => { + values + .iter_mut() + .for_each(|(_, inner)| inner.push(&Value::Null)); + target.push(false); + } + }; + }); + // Then deserialize each field's JSON values buffer to the appropriate Arrow2 array. + // + // Column ordering invariant - this assumes that values and target.mut_values() have aligned columns; + // we can assume this because: + // - target.mut_values() is guaranteed to have the same column ordering as target.data_type().fields, + // - values is an ordered map, whose ordering is tied to target.data_type().fields. + values + .into_values() + .zip(target.mut_values()) + .for_each(|(col_values, col_mut_arr)| deserialize_into(col_mut_arr, col_values.as_slice())); +} diff --git a/src/daft-json/src/inference.rs b/src/daft-json/src/inference.rs new file mode 100644 index 0000000000..4afac92836 --- /dev/null +++ b/src/daft-json/src/inference.rs @@ -0,0 +1,233 @@ +use std::{borrow::Borrow, collections::HashSet}; + +use arrow2::datatypes::{DataType, Field, Metadata, Schema, TimeUnit}; +use arrow2::error::{Error, Result}; +use indexmap::IndexMap; +use json_deserializer::{Number, Value}; + +const ITEM_NAME: &str = "item"; + +/// Infer Arrow2 schema from JSON Value record. +pub(crate) fn infer_records_schema(record: &Value) -> Result { + let fields = match record { + Value::Object(record) => record + .iter() + .map(|(name, value)| { + let data_type = infer(value)?; + + Ok(Field { + name: name.clone(), + data_type, + is_nullable: true, + metadata: Metadata::default(), + }) + }) + .collect::>>(), + _ => Err(Error::ExternalFormat( + "Deserialized JSON value is not an Object record".to_string(), + )), + }?; + + Ok(Schema { + fields, + metadata: Metadata::default(), + }) +} + +/// Infers [`DataType`] from [`Value`]. +fn infer(json: &Value) -> Result { + Ok(match json { + Value::Bool(_) => DataType::Boolean, + Value::Array(array) => infer_array(array)?, + Value::Null => DataType::Null, + Value::Number(number) => infer_number(number), + Value::String(string) => infer_string(string), + Value::Object(inner) => infer_object(inner)?, + }) +} + +fn infer_string(string: &str) -> DataType { + daft_decoding::inference::infer_string(string) +} + +fn infer_object(inner: &IndexMap) -> Result { + let fields = inner + .iter() + .map(|(key, value)| infer(value).map(|dt| (key, dt))) + .map(|maybe_dt| { + let (key, dt) = maybe_dt?; + Ok(Field::new(key, dt, true)) + }) + .collect::>>()?; + Ok(DataType::Struct(fields)) +} + +fn infer_array(values: &[Value]) -> Result { + let types = values + .iter() + .map(infer) + // Deduplicate dtypes. + .collect::>>()?; + + let dt = if !types.is_empty() { + coerce_data_type(types) + } else { + DataType::Null + }; + + // Don't add a record that contains only nulls. + Ok(if dt == DataType::Null { + dt + } else { + DataType::List(Box::new(Field::new(ITEM_NAME, dt, true))) + }) +} + +fn infer_number(n: &Number) -> DataType { + match n { + Number::Float(..) => DataType::Float64, + Number::Integer(..) => DataType::Int64, + } +} + +/// Convert each column's set of infered dtypes to a field with a consolidated dtype, following the coercion rules +/// defined in coerce_data_type. +pub(crate) fn column_types_map_to_fields( + column_types: IndexMap>, +) -> Vec { + column_types + .into_iter() + .map(|(name, dtype_set)| { + // Get consolidated dtype for column. + let dtype = coerce_data_type(dtype_set); + arrow2::datatypes::Field::new(name, dtype, true) + }) + .collect::>() +} + +/// Coerce an heterogeneous set of [`DataType`] into a single one. Rules: +/// * The empty set is coerced to `Null` +/// * `Int64` and `Float64` are `Float64` +/// * Lists and scalars are coerced to a list of a compatible scalar +/// * Structs contain the union of all fields +/// * All other types are coerced to `Utf8` +pub(crate) fn coerce_data_type(mut datatypes: HashSet) -> DataType { + // Drop null dtype from the dtype set. + datatypes.remove(&DataType::Null); + + if datatypes.is_empty() { + return DataType::Null; + } + + // All equal. + if datatypes.len() == 1 { + return datatypes.into_iter().next().unwrap(); + } + + let are_all_structs = datatypes + .iter() + .all(|x| matches!((*x).borrow(), DataType::Struct(_))); + + if are_all_structs { + // All structs => union of all field dtypes (these may have equal names). + let fields = datatypes.into_iter().fold(vec![], |mut acc, dt| { + if let DataType::Struct(new_fields) = dt { + acc.extend(new_fields); + }; + acc + }); + // Group fields by unique names. + let fields = fields.into_iter().fold( + IndexMap::>::new(), + |mut acc, field| { + match acc.entry(field.name) { + indexmap::map::Entry::Occupied(mut v) => { + v.get_mut().insert(field.data_type); + } + indexmap::map::Entry::Vacant(v) => { + let mut a = HashSet::new(); + a.insert(field.data_type); + v.insert(a); + } + } + acc + }, + ); + // Coerce dtype set for each field. + let fields = fields + .into_iter() + .map(|(name, dts)| Field::new(name, coerce_data_type(dts), true)) + .collect(); + return DataType::Struct(fields); + } + datatypes + .into_iter() + .reduce(|lhs, rhs| { + match (lhs, rhs) { + (lhs, rhs) if lhs == rhs => lhs, + (DataType::Utf8, _) | (_, DataType::Utf8) => DataType::Utf8, + (DataType::List(lhs), DataType::List(rhs)) => { + let inner = + coerce_data_type([lhs.data_type().clone(), rhs.data_type().clone()].into()); + DataType::List(Box::new(Field::new(ITEM_NAME, inner, true))) + } + (scalar, DataType::List(list)) | (DataType::List(list), scalar) => { + let inner = coerce_data_type([scalar, list.data_type().clone()].into()); + DataType::List(Box::new(Field::new(ITEM_NAME, inner, true))) + } + (DataType::Float64, DataType::Int64) | (DataType::Int64, DataType::Float64) => { + DataType::Float64 + } + (DataType::Int64, DataType::Boolean) | (DataType::Boolean, DataType::Int64) => { + DataType::Int64 + } + (DataType::Time32(left_tu), DataType::Time32(right_tu)) => { + // Set unified time unit to the lowest granularity time unit. + let unified_tu = if left_tu == right_tu + || time_unit_to_ordinal(&left_tu) < time_unit_to_ordinal(&right_tu) + { + left_tu + } else { + right_tu + }; + DataType::Time32(unified_tu) + } + ( + DataType::Timestamp(left_tu, left_tz), + DataType::Timestamp(right_tu, right_tz), + ) => { + // Set unified time unit to the lowest granularity time unit. + let unified_tu = if left_tu == right_tu + || time_unit_to_ordinal(&left_tu) < time_unit_to_ordinal(&right_tu) + { + left_tu + } else { + right_tu + }; + // Set unified time zone to UTC. + let unified_tz = match (&left_tz, &right_tz) { + (None, None) => None, + (None, _) | (_, None) => return DataType::Utf8, + (Some(l), Some(r)) if l == r => left_tz, + (Some(_), Some(_)) => Some("Z".to_string()), + }; + DataType::Timestamp(unified_tu, unified_tz) + } + (DataType::Timestamp(_, None), DataType::Date32) + | (DataType::Date32, DataType::Timestamp(_, None)) => { + DataType::Timestamp(TimeUnit::Second, None) + } + (_, _) => DataType::Utf8, + } + }) + .unwrap() +} + +fn time_unit_to_ordinal(tu: &TimeUnit) -> usize { + match tu { + TimeUnit::Second => 0, + TimeUnit::Millisecond => 1, + TimeUnit::Microsecond => 2, + TimeUnit::Nanosecond => 3, + } +} diff --git a/src/daft-json/src/lib.rs b/src/daft-json/src/lib.rs new file mode 100644 index 0000000000..667a81469c --- /dev/null +++ b/src/daft-json/src/lib.rs @@ -0,0 +1,73 @@ +#![feature(async_closure)] +#![feature(let_chains)] +#![feature(trait_alias)] +#![feature(trait_upcasting)] +use common_error::DaftError; +use futures::stream::TryChunksError; +use snafu::Snafu; + +mod decoding; +mod inference; +pub mod options; +#[cfg(feature = "python")] +pub mod python; +pub mod read; +pub mod schema; + +// pub use metadata::read_json_schema_bulk; +pub use options::{JsonConvertOptions, JsonParseOptions, JsonReadOptions}; +#[cfg(feature = "python")] +use pyo3::prelude::*; +pub use read::{read_json, read_json_bulk}; + +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("{source}"))] + IOError { source: daft_io::Error }, + #[snafu(display("{source}"))] + StdIOError { source: std::io::Error }, + #[snafu(display("{source}"))] + ArrowError { source: arrow2::error::Error }, + #[snafu(display("JSON deserialization error: {}", string))] + JsonDeserializationError { string: String }, + #[snafu(display("Error chunking: {}", source))] + ChunkError { + source: TryChunksError, + }, + #[snafu(display("Error joining spawned task: {}", source))] + JoinError { source: tokio::task::JoinError }, + #[snafu(display( + "Sender of OneShot Channel Dropped before sending data over: {}", + source + ))] + OneShotRecvError { + source: tokio::sync::oneshot::error::RecvError, + }, +} + +impl From for DaftError { + fn from(err: Error) -> DaftError { + match err { + Error::IOError { source } => source.into(), + _ => DaftError::External(err.into()), + } + } +} + +impl From for Error { + fn from(err: daft_io::Error) -> Self { + Error::IOError { source: err } + } +} + +type Result = std::result::Result; + +#[cfg(feature = "python")] +pub fn register_modules(_py: Python, parent: &PyModule) -> PyResult<()> { + parent.add_class::()?; + parent.add_class::()?; + parent.add_class::()?; + parent.add_wrapped(wrap_pyfunction!(python::pylib::read_json))?; + parent.add_wrapped(wrap_pyfunction!(python::pylib::read_json_schema))?; + Ok(()) +} diff --git a/src/daft-json/src/options.rs b/src/daft-json/src/options.rs new file mode 100644 index 0000000000..d897c25db2 --- /dev/null +++ b/src/daft-json/src/options.rs @@ -0,0 +1,212 @@ +use daft_core::{impl_bincode_py_state_serialization, schema::SchemaRef}; +use serde::{Deserialize, Serialize}; +#[cfg(feature = "python")] +use { + daft_core::python::schema::PySchema, + pyo3::{ + pyclass, pyclass::CompareOp, pymethods, types::PyBytes, PyObject, PyResult, PyTypeInfo, + Python, ToPyObject, + }, +}; + +/// Options for converting JSON data to Daft data. +#[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] +#[cfg_attr(feature = "python", pyclass(module = "daft.daft"))] +pub struct JsonConvertOptions { + pub limit: Option, + pub include_columns: Option>, + pub schema: Option, +} + +impl JsonConvertOptions { + pub fn new_internal( + limit: Option, + include_columns: Option>, + schema: Option, + ) -> Self { + Self { + limit, + include_columns, + schema, + } + } + + pub fn with_limit(self, limit: Option) -> Self { + Self { limit, ..self } + } + + pub fn with_include_columns(self, include_columns: Option>) -> Self { + Self { + include_columns, + ..self + } + } + + pub fn with_schema(self, schema: Option) -> Self { + Self { schema, ..self } + } +} + +impl Default for JsonConvertOptions { + fn default() -> Self { + Self::new_internal(None, None, None) + } +} + +#[cfg(feature = "python")] +#[pymethods] +impl JsonConvertOptions { + /// Create conversion options for the JSON reader. + /// + /// # Arguments: + /// + /// * `limit` - Only read this many rows. + /// * `include_columns` - The names of the columns that should be kept, e.g. via a projection. + /// * `schema` - The names and dtypes for the JSON columns. + #[new] + #[pyo3(signature = (limit=None, include_columns=None, schema=None))] + pub fn new( + limit: Option, + include_columns: Option>, + schema: Option, + ) -> Self { + Self::new_internal(limit, include_columns, schema.map(|s| s.into())) + } + + #[getter] + pub fn get_limit(&self) -> PyResult> { + Ok(self.limit) + } + + #[getter] + pub fn get_include_columns(&self) -> PyResult>> { + Ok(self.include_columns.clone()) + } + + #[getter] + pub fn get_schema(&self) -> PyResult> { + Ok(self.schema.as_ref().map(|s| s.clone().into())) + } + + fn __richcmp__(&self, other: &Self, op: CompareOp) -> bool { + match op { + CompareOp::Eq => self == other, + CompareOp::Ne => !self.__richcmp__(other, CompareOp::Eq), + _ => unimplemented!("not implemented"), + } + } + + pub fn __str__(&self) -> PyResult { + Ok(format!("{:?}", self)) + } +} + +impl_bincode_py_state_serialization!(JsonConvertOptions); + +/// Options for parsing JSON files. +#[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] +#[cfg_attr(feature = "python", pyclass(module = "daft.daft", get_all))] +pub struct JsonParseOptions {} + +impl JsonParseOptions { + pub fn new_internal() -> Self { + Self {} + } +} + +impl Default for JsonParseOptions { + fn default() -> Self { + Self::new_internal() + } +} + +#[cfg(feature = "python")] +#[pymethods] +impl JsonParseOptions { + /// Create parsing options for the JSON reader. + #[new] + pub fn new() -> PyResult { + Ok(Self::new_internal()) + } + + fn __richcmp__(&self, other: &Self, op: CompareOp) -> bool { + match op { + CompareOp::Eq => self == other, + CompareOp::Ne => !self.__richcmp__(other, CompareOp::Eq), + _ => unimplemented!("not implemented"), + } + } + + pub fn __str__(&self) -> PyResult { + Ok(format!("{:?}", self)) + } +} + +impl_bincode_py_state_serialization!(JsonParseOptions); + +/// Options for reading JSON files. +#[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] +#[cfg_attr(feature = "python", pyclass(module = "daft.daft", get_all))] +pub struct JsonReadOptions { + pub buffer_size: Option, + pub chunk_size: Option, +} + +impl JsonReadOptions { + pub fn new_internal(buffer_size: Option, chunk_size: Option) -> Self { + Self { + buffer_size, + chunk_size, + } + } + + pub fn with_buffer_size(self, buffer_size: Option) -> Self { + Self { + buffer_size, + chunk_size: self.chunk_size, + } + } + + pub fn with_chunk_size(self, chunk_size: Option) -> Self { + Self { + buffer_size: self.buffer_size, + chunk_size, + } + } +} + +impl Default for JsonReadOptions { + fn default() -> Self { + Self::new_internal(None, None) + } +} + +#[cfg(feature = "python")] +#[pymethods] +impl JsonReadOptions { + /// Create reading options for the JSON reader. + /// + /// # Arguments: + /// + /// * `buffer_size` - Size of the buffer (in bytes) used by the streaming reader. + /// * `chunk_size` - Size of the chunks (in bytes) deserialized in parallel by the streaming reader. + #[new] + #[pyo3(signature = (buffer_size=None, chunk_size=None))] + pub fn new(buffer_size: Option, chunk_size: Option) -> Self { + Self::new_internal(buffer_size, chunk_size) + } + + fn __richcmp__(&self, other: &Self, op: CompareOp) -> bool { + match op { + CompareOp::Eq => self == other, + CompareOp::Ne => !self.__richcmp__(other, CompareOp::Eq), + _ => unimplemented!("not implemented"), + } + } + + pub fn __str__(&self) -> PyResult { + Ok(format!("{:?}", self)) + } +} + +impl_bincode_py_state_serialization!(JsonReadOptions); diff --git a/src/daft-json/src/python.rs b/src/daft-json/src/python.rs new file mode 100644 index 0000000000..139ba4a531 --- /dev/null +++ b/src/daft-json/src/python.rs @@ -0,0 +1,70 @@ +pub mod pylib { + use std::sync::Arc; + + use daft_core::python::schema::PySchema; + use daft_io::{get_io_client, python::IOConfig, IOStatsContext}; + use daft_table::python::PyTable; + use pyo3::{pyfunction, PyResult, Python}; + + use crate::{JsonConvertOptions, JsonParseOptions, JsonReadOptions}; + + #[allow(clippy::too_many_arguments)] + #[pyfunction] + pub fn read_json( + py: Python, + uri: &str, + convert_options: Option, + parse_options: Option, + read_options: Option, + io_config: Option, + multithreaded_io: Option, + max_chunks_in_flight: Option, + ) -> PyResult { + py.allow_threads(|| { + let io_stats = IOStatsContext::new(format!("read_json: for uri {uri}")); + + let io_client = get_io_client( + multithreaded_io.unwrap_or(true), + io_config.unwrap_or_default().config.into(), + )?; + Ok(crate::read::read_json( + uri, + convert_options, + parse_options, + read_options, + io_client, + Some(io_stats), + multithreaded_io.unwrap_or(true), + max_chunks_in_flight, + )? + .into()) + }) + } + + #[pyfunction] + pub fn read_json_schema( + py: Python, + uri: &str, + parse_options: Option, + max_bytes: Option, + io_config: Option, + multithreaded_io: Option, + ) -> PyResult { + py.allow_threads(|| { + let io_stats = IOStatsContext::new(format!("read_json_schema: for uri {uri}")); + + let io_client = get_io_client( + multithreaded_io.unwrap_or(true), + io_config.unwrap_or_default().config.into(), + )?; + let schema = crate::schema::read_json_schema( + uri, + parse_options, + max_bytes, + io_client, + Some(io_stats), + )?; + Ok(Arc::new(schema).into()) + }) + } +} diff --git a/src/daft-json/src/read.rs b/src/daft-json/src/read.rs new file mode 100644 index 0000000000..afbdefa13b --- /dev/null +++ b/src/daft-json/src/read.rs @@ -0,0 +1,1218 @@ +use std::{collections::HashMap, num::NonZeroUsize, sync::Arc}; + +use common_error::{DaftError, DaftResult}; +use daft_core::{schema::Schema, utils::arrow::cast_array_for_daft_if_needed, Series}; +use daft_io::{get_runtime, GetResult, IOClient, IOStatsRef}; +use daft_table::Table; +use futures::{Stream, StreamExt, TryStreamExt}; +use rayon::prelude::{IndexedParallelIterator, IntoParallelIterator, ParallelIterator}; +use snafu::{ + futures::{try_future::Context, TryFutureExt, TryStreamExt as _}, + ResultExt, +}; +use tokio::{ + fs::File, + io::{AsyncBufRead, AsyncBufReadExt, BufReader}, + task::JoinHandle, +}; +use tokio_util::io::StreamReader; + +use crate::{decoding::deserialize_records, ArrowSnafu, ChunkSnafu}; +use crate::{ + schema::read_json_schema_single, JsonConvertOptions, JsonParseOptions, JsonReadOptions, +}; +use daft_compression::CompressionCodec; + +trait LineChunkStream = Stream>>; +trait ColumnArrayChunkStream = Stream< + Item = super::Result< + Context< + JoinHandle>>>, + super::JoinSnafu, + super::Error, + >, + >, +>; + +#[allow(clippy::too_many_arguments)] +pub fn read_json( + uri: &str, + convert_options: Option, + parse_options: Option, + read_options: Option, + io_client: Arc, + io_stats: Option, + multithreaded_io: bool, + max_chunks_in_flight: Option, +) -> DaftResult { + let runtime_handle = get_runtime(multithreaded_io)?; + let _rt_guard = runtime_handle.enter(); + runtime_handle.block_on(async { + read_json_single_into_table( + uri, + convert_options, + parse_options, + read_options, + io_client, + io_stats, + max_chunks_in_flight, + ) + .await + }) +} + +#[allow(clippy::too_many_arguments)] +pub fn read_json_bulk( + uris: &[&str], + convert_options: Option, + parse_options: Option, + read_options: Option, + io_client: Arc, + io_stats: Option, + multithreaded_io: bool, + max_chunks_in_flight: Option, + num_parallel_tasks: usize, +) -> DaftResult> { + let runtime_handle = get_runtime(multithreaded_io)?; + let _rt_guard = runtime_handle.enter(); + let tables = runtime_handle.block_on(async move { + // Launch a read task per URI, throttling the number of concurrent file reads to num_parallel tasks. + let task_stream = futures::stream::iter(uris.iter().map(|uri| { + let (uri, convert_options, parse_options, read_options, io_client, io_stats) = ( + uri.to_string(), + convert_options.clone(), + parse_options.clone(), + read_options.clone(), + io_client.clone(), + io_stats.clone(), + ); + tokio::task::spawn(async move { + let table = read_json_single_into_table( + uri.as_str(), + convert_options, + parse_options, + read_options, + io_client, + io_stats, + max_chunks_in_flight, + ) + .await?; + DaftResult::Ok(table) + }) + .context(crate::JoinSnafu) + })); + let mut remaining_rows = convert_options + .as_ref() + .and_then(|opts| opts.limit.map(|limit| limit as i64)); + task_stream + // Limit the number of file reads we have in flight at any given time. + .buffered(num_parallel_tasks) + // Terminate the stream if we have already reached the row limit. With the upstream buffering, we will still read up to + // num_parallel_tasks redundant files. + .try_take_while(|result| { + match (result, remaining_rows) { + // Limit has been met, early-teriminate. + (_, Some(rows_left)) if rows_left <= 0 => futures::future::ready(Ok(false)), + // Limit has not yet been met, update remaining limit slack and continue. + (Ok(table), Some(rows_left)) => { + remaining_rows = Some(rows_left - table.len() as i64); + futures::future::ready(Ok(true)) + } + // (1) No limit, never early-terminate. + // (2) Encountered error, propagate error to try_collect to allow it to short-circuit. + (_, None) | (Err(_), _) => futures::future::ready(Ok(true)), + } + }) + .try_collect::>() + .await + })?; + tables.into_iter().collect::>>() +} + +async fn read_json_single_into_table( + uri: &str, + convert_options: Option, + parse_options: Option, + read_options: Option, + io_client: Arc, + io_stats: Option, + max_chunks_in_flight: Option, +) -> DaftResult
{ + let (chunk_stream, schema) = read_json_single_into_stream( + uri, + convert_options.unwrap_or_default(), + parse_options.unwrap_or_default(), + read_options, + io_client, + io_stats, + ) + .await?; + // Default max chunks in flight is set to 2x the number of cores, which should ensure pipelining of reading chunks + // with the parsing of chunks on the rayon threadpool. + let max_chunks_in_flight = max_chunks_in_flight.unwrap_or_else(|| { + std::thread::available_parallelism() + .unwrap_or(NonZeroUsize::new(2).unwrap()) + .checked_mul(2.try_into().unwrap()) + .unwrap() + .try_into() + .unwrap() + }); + // Collect all chunks in chunk x column form. + let chunks = chunk_stream + // Limit the number of chunks we have in flight at any given time. + .try_buffered(max_chunks_in_flight) + .try_collect::>() + .await? + .into_iter() + .collect::>>()?; + // Handle empty table case. + if chunks.is_empty() { + let daft_schema = Arc::new(Schema::try_from(&schema)?); + return Table::empty(Some(daft_schema)); + } + // Transpose chunk x column into column x chunk. + let mut column_arrays = vec![Vec::with_capacity(chunks.len()); chunks[0].len()]; + for chunk in chunks.into_iter() { + for (idx, col) in chunk.into_iter().enumerate() { + column_arrays[idx].push(col); + } + } + // Build table from chunks. + // TODO(Clark): Don't concatenate all chunks from a file into a single table, since MicroPartition is natively chunked. + chunks_to_table(column_arrays, schema) +} + +async fn read_json_single_into_stream( + uri: &str, + convert_options: JsonConvertOptions, + parse_options: JsonParseOptions, + read_options: Option, + io_client: Arc, + io_stats: Option, +) -> DaftResult<( + impl ColumnArrayChunkStream + Send, + arrow2::datatypes::Schema, +)> { + let schema = match convert_options.schema { + Some(schema) => schema.to_arrow()?, + None => read_json_schema_single( + uri, + parse_options.clone(), + // Read at most 1 MiB when doing schema inference. + Some(1024 * 1024), + io_client.clone(), + io_stats.clone(), + ) + .await? + .to_arrow()?, + }; + let (reader, buffer_size, chunk_size): (Box, usize, usize) = + match io_client + .single_url_get(uri.to_string(), None, io_stats) + .await? + { + GetResult::File(file) => { + // Use user-provided buffer size, falling back to 8 * the user-provided chunk size if that exists, otherwise falling back to 512 KiB as the default. + let buffer_size = read_options + .as_ref() + .and_then(|opt| { + opt.buffer_size + .or_else(|| opt.chunk_size.map(|cs| (64 * cs).min(256 * 1024 * 1024))) + }) + .unwrap_or(256 * 1024); + ( + Box::new(BufReader::with_capacity( + buffer_size, + File::open(file.path).await?, + )), + buffer_size, + read_options + .as_ref() + .and_then(|opt| { + opt.chunk_size + .or_else(|| opt.buffer_size.map(|bs| (bs / 64).max(16))) + }) + .unwrap_or(64), + ) + } + GetResult::Stream(stream, _, _) => ( + Box::new(StreamReader::new(stream)), + // Use user-provided buffer size, falling back to 8 * the user-provided chunk size if that exists, otherwise falling back to 512 KiB as the default. + read_options + .as_ref() + .and_then(|opt| { + opt.buffer_size + .or_else(|| opt.chunk_size.map(|cs| (256 * cs).min(256 * 1024 * 1024))) + }) + .unwrap_or(8 * 1024 * 1024), + read_options + .as_ref() + .and_then(|opt| { + opt.chunk_size + .or_else(|| opt.buffer_size.map(|bs| (bs / 256).max(16))) + }) + .unwrap_or(64), + ), + }; + // If file is compressed, wrap stream in decoding stream. + let reader: Box = match CompressionCodec::from_uri(uri) { + Some(compression) => Box::new(tokio::io::BufReader::with_capacity( + buffer_size, + compression.to_decoder(reader), + )), + None => reader, + }; + let read_stream = read_into_line_chunk_stream(reader, convert_options.limit, chunk_size); + let (projected_schema, schema_is_projection) = match convert_options.include_columns { + Some(projection) => { + let mut field_map = schema + .fields + .into_iter() + .map(|f| (f.name.clone(), f)) + .collect::>(); + let projected_fields = projection.into_iter().map(|col| field_map.remove(col.as_str()).ok_or(DaftError::ValueError(format!("Column {} in the projection doesn't exist in the JSON file; existing columns = {:?}", col, field_map.keys())))).collect::>>()?; + ( + arrow2::datatypes::Schema::from(projected_fields).with_metadata(schema.metadata), + true, + ) + } + None => (schema, false), + }; + Ok(( + parse_into_column_array_chunk_stream( + read_stream, + projected_schema.clone().into(), + schema_is_projection, + ), + projected_schema, + )) +} + +fn read_into_line_chunk_stream( + reader: R, + num_rows: Option, + chunk_size: usize, +) -> impl LineChunkStream + Send +where + R: AsyncBufRead + Unpin + Send + 'static, +{ + let num_rows = num_rows.unwrap_or(usize::MAX); + // Stream of unparsed json string record chunks. + let line_stream = tokio_stream::wrappers::LinesStream::new(reader.lines()); + line_stream + .take(num_rows) + .try_chunks(chunk_size) + .context(ChunkSnafu) +} + +fn parse_into_column_array_chunk_stream( + stream: impl LineChunkStream + Send, + schema: Arc, + schema_is_projection: bool, +) -> impl ColumnArrayChunkStream + Send { + // Parsing stream: we spawn background tokio + rayon tasks so we can pipeline chunk parsing with chunk reading, and + // we further parse each chunk column in parallel on the rayon threadpool. + stream.map_ok(move |records| { + let schema = schema.clone(); + tokio::spawn(async move { + let (send, recv) = tokio::sync::oneshot::channel(); + rayon::spawn(move || { + let result = (move || { + // TODO(Clark): Switch to streaming parse + array construction? + let parsed = records + .iter() + .map(|unparsed_record| { + json_deserializer::parse(unparsed_record.as_bytes()).map_err(|e| { + super::Error::JsonDeserializationError { + string: e.to_string(), + } + }) + }) + .collect::>>()?; + let chunk = deserialize_records(parsed, schema.as_ref(), schema_is_projection) + .context(ArrowSnafu)?; + Ok(chunk) + })(); + let _ = send.send(result); + }); + recv.await.context(super::OneShotRecvSnafu {})? + }) + .context(super::JoinSnafu {}) + }) +} + +fn chunks_to_table( + chunks: Vec>>, + schema: arrow2::datatypes::Schema, +) -> DaftResult
{ + // Concatenate column chunks and convert into Daft Series. + // Note that this concatenation is done in parallel on the rayon threadpool. + let columns_series = chunks + .into_par_iter() + .zip(&schema.fields) + .map(|(mut arrays, field)| { + let array = if arrays.len() > 1 { + // Concatenate all array chunks. + let unboxed_arrays = arrays.iter().map(Box::as_ref).collect::>(); + arrow2::compute::concatenate::concatenate(unboxed_arrays.as_slice())? + } else { + // Return single array chunk directly. + arrays.pop().unwrap() + }; + Series::try_from((field.name.as_ref(), cast_array_for_daft_if_needed(array))) + }) + .collect::>>()?; + // Build Daft Table. + let daft_schema = Schema::try_from(&schema)?; + Table::new(daft_schema, columns_series) +} + +#[cfg(test)] +mod tests { + use std::{collections::HashSet, io::BufRead, sync::Arc}; + + use common_error::DaftResult; + + use daft_core::{ + datatypes::{Field, TimeUnit}, + schema::Schema, + utils::arrow::{cast_array_for_daft_if_needed, cast_array_from_daft_if_needed}, + DataType, + }; + use daft_io::{IOClient, IOConfig}; + use daft_table::Table; + use indexmap::IndexMap; + use rstest::rstest; + + use crate::{ + decoding::deserialize_records, + inference::{column_types_map_to_fields, infer_records_schema}, + }; + use crate::{JsonConvertOptions, JsonReadOptions}; + + use super::read_json; + + fn check_equal_local_arrow2( + path: &str, + out: &Table, + limit: Option, + projection: Option>, + ) { + let reader = std::io::BufReader::new(std::fs::File::open(path).unwrap()); + let lines = reader.lines().collect::>(); + let parsed = lines + .iter() + .take(limit.unwrap_or(usize::MAX)) + .map(|record| json_deserializer::parse(record.as_ref().unwrap().as_bytes()).unwrap()) + .collect::>(); + // Get consolidated schema from parsed JSON. + let mut column_types: IndexMap> = + IndexMap::new(); + parsed.iter().for_each(|record| { + let schema = infer_records_schema(record).unwrap(); + for field in schema.fields { + match column_types.entry(field.name) { + indexmap::map::Entry::Occupied(mut v) => { + v.get_mut().insert(field.data_type); + } + indexmap::map::Entry::Vacant(v) => { + let mut a = HashSet::new(); + a.insert(field.data_type); + v.insert(a); + } + } + } + }); + let fields = column_types_map_to_fields(column_types); + let schema: arrow2::datatypes::Schema = fields.into(); + // Apply projection to schema. + let mut field_map = schema + .fields + .iter() + .map(|f| (f.name.clone(), f.clone())) + .collect::>(); + let (schema, is_projection) = match &projection { + Some(projection) => ( + projection + .iter() + .map(|c| field_map.remove(c.as_str()).unwrap()) + .collect::>() + .into(), + true, + ), + None => ( + field_map + .into_values() + .into_iter() + .collect::>() + .into(), + false, + ), + }; + // Deserialize JSON records into Arrow2 column arrays. + let columns = deserialize_records(parsed, &schema, is_projection).unwrap(); + // Roundtrip columns with Daft for casting. + let columns = columns + .into_iter() + .map(|c| cast_array_from_daft_if_needed(cast_array_for_daft_if_needed(c))) + .collect::>(); + // Roundtrip schema with Daft for casting. + let schema = Schema::try_from(&schema).unwrap().to_arrow().unwrap(); + assert_eq!(out.schema.to_arrow().unwrap(), schema); + let out_columns = (0..out.num_columns()) + .map(|i| out.get_column_by_index(i).unwrap().to_arrow()) + .collect::>(); + assert_eq!(out_columns, columns); + } + + #[rstest] + fn test_json_read_local( + #[values( + // Uncompressed + None, + // brotli + Some("br"), + // bzip2 + Some("bz2"), + // TODO(Clark): Add deflate compressed JSON file to test data fixtures. + // // deflate + // Some("deflate"), + // gzip + Some("gz"), + // lzma + Some("lzma"), + // xz + Some("xz"), + // zlib + Some("zl"), + // zstd + Some("zst"), + )] + compression: Option<&str>, + ) -> DaftResult<()> { + let file = format!( + "{}/test/iris_tiny.jsonl{}", + env!("CARGO_MANIFEST_DIR"), + compression.map_or("".to_string(), |ext| format!(".{}", ext)) + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json(file.as_ref(), None, None, None, io_client, None, true, None)?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + if compression.is_none() { + check_equal_local_arrow2(file.as_ref(), &table, None, None); + } + + Ok(()) + } + + #[rstest] + fn test_json_read_local_dtypes() -> DaftResult<()> { + let file = format!("{}/test/dtypes.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json(file.as_ref(), None, None, None, io_client, None, true, None)?; + assert_eq!(table.len(), 4); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("int", DataType::Int64), + Field::new("float", DataType::Float64), + Field::new("bool", DataType::Boolean), + Field::new("str", DataType::Utf8), + Field::new("null", DataType::Null), + Field::new("date", DataType::Date), + // TODO(Clark): Add coverage for time parsing once we add support for representing time series in Daft. + // // Time unit should be coarest granularity found in file, i.e. seconds. + // Field::new("time", DataType::Time(TimeUnit::Nanoseconds)), + // Time unit should be coarsest granularity found in file, i.e. seconds due to naive date inclusion. + Field::new( + "naive_timestamp", + DataType::Timestamp(TimeUnit::Seconds, None) + ), + // Timezone should be UTC due to field having multiple different timezones across records. + // Time unit should be coarsest granularity found in file, i.e. milliseconds. + Field::new( + "timestamp", + DataType::Timestamp(TimeUnit::Milliseconds, Some("Z".to_string())) + ), + Field::new("list", DataType::List(Box::new(DataType::Int64))), + Field::new( + "obj", + DataType::Struct(vec![ + Field::new("a", DataType::Int64), + Field::new("b", DataType::Boolean) + ]) + ), + Field::new( + "nested_list", + DataType::List(Box::new(DataType::List(Box::new(DataType::Struct(vec![ + Field::new("a", DataType::Utf8), + ]))))) + ), + Field::new( + "nested_obj", + DataType::Struct(vec![ + Field::new( + "obj", + DataType::Struct(vec![Field::new("a", DataType::Int64)]) + ), + Field::new("list", DataType::List(Box::new(DataType::Int64))), + ]) + ), + ])? + .into(), + ); + check_equal_local_arrow2(file.as_ref(), &table, None, None); + + Ok(()) + } + + #[test] + fn test_json_read_local_limit() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + Some(JsonConvertOptions::default().with_limit(Some(5))), + None, + None, + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 5); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + check_equal_local_arrow2(file.as_ref(), &table, Some(5), None); + + Ok(()) + } + + #[test] + fn test_json_read_local_projection() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + Some( + JsonConvertOptions::default().with_include_columns(Some(vec![ + "petalWidth".to_string(), + "petalLength".to_string(), + ])), + ), + None, + None, + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("petalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + ])? + .into(), + ); + check_equal_local_arrow2( + file.as_ref(), + &table, + None, + Some(vec!["petalWidth".to_string(), "petalLength".to_string()]), + ); + + Ok(()) + } + + #[test] + fn test_json_read_local_larger_than_buffer_size() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + None, + None, + Some(JsonReadOptions::default().with_buffer_size(Some(128))), + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + check_equal_local_arrow2(file.as_ref(), &table, None, None); + + Ok(()) + } + + #[test] + fn test_json_read_local_larger_than_chunk_size() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + None, + None, + Some(JsonReadOptions::default().with_chunk_size(Some(5))), + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + check_equal_local_arrow2(file.as_ref(), &table, None, None); + + Ok(()) + } + + #[test] + fn test_json_read_local_throttled_streaming() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + None, + None, + Some(JsonReadOptions::default().with_chunk_size(Some(5))), + io_client, + None, + true, + Some(2), + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + check_equal_local_arrow2(file.as_ref(), &table, None, None); + + Ok(()) + } + + #[test] + fn test_json_read_local_nulls() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny_nulls.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json(file.as_ref(), None, None, None, io_client, None, true, None)?; + assert_eq!(table.len(), 6); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + check_equal_local_arrow2(file.as_ref(), &table, None, None); + + Ok(()) + } + + #[test] + fn test_json_read_local_all_null_column() -> DaftResult<()> { + let file = format!( + "{}/test/iris_tiny_all_null_column.jsonl", + env!("CARGO_MANIFEST_DIR"), + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json(file.as_ref(), None, None, None, io_client, None, true, None)?; + assert_eq!(table.len(), 6); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + // All null column parsed as null dtype. + Field::new("petalLength", DataType::Null), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + let null_column = table.get_column("petalLength")?; + assert_eq!(null_column.data_type(), &DataType::Null); + assert_eq!(null_column.len(), 6); + assert_eq!( + null_column.to_arrow(), + Box::new(arrow2::array::NullArray::new( + arrow2::datatypes::DataType::Null, + 6 + )) as Box + ); + + Ok(()) + } + + #[test] + fn test_json_read_local_all_null_column_with_schema() -> DaftResult<()> { + let file = format!( + "{}/test/iris_tiny_all_null_column.jsonl", + env!("CARGO_MANIFEST_DIR"), + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + // Manually specify this column as all-null. + Field::new("petalLength", DataType::Null), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])?; + let table = read_json( + file.as_ref(), + Some(JsonConvertOptions::default().with_schema(Some(schema.into()))), + None, + None, + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 6); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + // All null column parsed as null dtype. + Field::new("petalLength", DataType::Null), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + let null_column = table.get_column("petalLength")?; + assert_eq!(null_column.data_type(), &DataType::Null); + assert_eq!(null_column.len(), 6); + assert_eq!( + null_column.to_arrow(), + Box::new(arrow2::array::NullArray::new( + arrow2::datatypes::DataType::Null, + 6 + )) as Box + ); + + Ok(()) + } + + #[test] + fn test_json_read_local_all_null_column_with_schema_well_typed() -> DaftResult<()> { + let file = format!( + "{}/test/iris_tiny_all_null_column.jsonl", + env!("CARGO_MANIFEST_DIR"), + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + let schema = Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + // Provide a manual type for the all-null column. + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])?; + + let table = read_json( + file.as_ref(), + Some(JsonConvertOptions::default().with_schema(Some(schema.into()))), + None, + None, + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 6); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + // All null column should have hte provided dtype. + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + let null_column = table.get_column("petalLength")?; + assert_eq!(null_column.data_type(), &DataType::Float64); + assert_eq!(null_column.len(), 6); + assert_eq!(null_column.to_arrow().null_count(), 6); + + Ok(()) + } + + #[test] + fn test_json_read_local_wrong_type_yields_nulls() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = Schema::new(vec![ + // Conversion to all of these types should fail, resulting in nulls. + Field::new("sepalLength", DataType::Boolean), + Field::new("sepalWidth", DataType::Boolean), + Field::new("petalLength", DataType::Boolean), + Field::new("petalWidth", DataType::Boolean), + Field::new("species", DataType::Int64), + ])?; + let table = read_json( + file.as_ref(), + Some(JsonConvertOptions::default().with_schema(Some(schema.into()))), + None, + None, + io_client, + None, + true, + None, + )?; + let num_rows = table.len(); + assert_eq!(num_rows, 20); + // Check that all columns are all null. + for idx in 0..table.num_columns() { + let column = table.get_column_by_index(idx)?; + assert_eq!(column.to_arrow().null_count(), num_rows); + } + + Ok(()) + } + + #[rstest] + fn test_json_read_s3( + #[values( + // Uncompressed + None, + // brotli + Some("br"), + // bzip2 + Some("bz2"), + // TODO(Clark): Add deflate compressed JSON file to test data fixtures. + // // deflate + // Some("deflate"), + // gzip + Some("gz"), + // lzma + Some("lzma"), + // xz + Some("xz"), + // zlib + Some("zl"), + // zstd + Some("zst"), + )] + compression: Option<&str>, + ) -> DaftResult<()> { + let file = format!( + "s3://daft-public-data/test_fixtures/json-dev/iris_tiny.jsonl{}", + compression.map_or("".to_string(), |ext| format!(".{}", ext)) + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json(file.as_ref(), None, None, None, io_client, None, true, None)?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + + Ok(()) + } + + #[test] + fn test_json_read_s3_limit() -> DaftResult<()> { + let file = "s3://daft-public-data/test_fixtures/json-dev/iris_tiny.jsonl"; + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + Some(JsonConvertOptions::default().with_limit(Some(5))), + None, + None, + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 5); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + + Ok(()) + } + + #[test] + fn test_json_read_s3_projection() -> DaftResult<()> { + let file = "s3://daft-public-data/test_fixtures/json-dev/iris_tiny.jsonl"; + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + Some( + JsonConvertOptions::default().with_include_columns(Some(vec![ + "petalWidth".to_string(), + "petalLength".to_string(), + ])), + ), + None, + None, + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("petalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + ])? + .into(), + ); + + Ok(()) + } + + #[test] + fn test_json_read_s3_larger_than_buffer_size() -> DaftResult<()> { + let file = "s3://daft-public-data/test_fixtures/json-dev/iris_tiny.jsonl"; + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + None, + None, + Some(JsonReadOptions::default().with_buffer_size(Some(128))), + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + + Ok(()) + } + + #[test] + fn test_json_read_s3_larger_than_chunk_size() -> DaftResult<()> { + let file = "s3://daft-public-data/test_fixtures/json-dev/iris_tiny.jsonl"; + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + None, + None, + Some(JsonReadOptions::default().with_chunk_size(Some(5))), + io_client, + None, + true, + None, + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + + Ok(()) + } + + #[test] + fn test_json_read_s3_throttled_streaming() -> DaftResult<()> { + let file = "s3://daft-public-data/test_fixtures/json-dev/iris_tiny.jsonl"; + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let table = read_json( + file.as_ref(), + None, + None, + Some(JsonReadOptions::default().with_chunk_size(Some(5))), + io_client, + None, + true, + Some(2), + )?; + assert_eq!(table.len(), 20); + assert_eq!( + table.schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + .into(), + ); + + Ok(()) + } +} diff --git a/src/daft-json/src/schema.rs b/src/daft-json/src/schema.rs new file mode 100644 index 0000000000..4b0c1c4c75 --- /dev/null +++ b/src/daft-json/src/schema.rs @@ -0,0 +1,449 @@ +use std::{collections::HashSet, sync::Arc}; + +use common_error::DaftResult; +use daft_core::schema::Schema; +use daft_io::{get_runtime, GetResult, IOClient, IOStatsRef}; +use futures::{StreamExt, TryStreamExt}; +use indexmap::IndexMap; +use json_deserializer::parse; +use snafu::ResultExt; +use tokio::{ + fs::File, + io::{AsyncBufRead, AsyncBufReadExt, BufReader}, +}; +use tokio_util::io::StreamReader; + +use crate::{ + inference::{column_types_map_to_fields, infer_records_schema}, + ArrowSnafu, JsonParseOptions, StdIOSnafu, +}; +use daft_compression::CompressionCodec; + +#[derive(Debug, Clone)] +pub struct JsonReadStats { + pub total_bytes_read: usize, + pub total_records_read: usize, + pub mean_record_size_bytes: f64, + pub stddev_record_size_bytes: f64, +} + +impl JsonReadStats { + pub fn new( + total_bytes_read: usize, + total_records_read: usize, + mean_record_size_bytes: f64, + stddev_record_size_bytes: f64, + ) -> Self { + Self { + total_bytes_read, + total_records_read, + mean_record_size_bytes, + stddev_record_size_bytes, + } + } +} + +impl Default for JsonReadStats { + fn default() -> Self { + Self::new(0, 0, 0f64, 0f64) + } +} + +pub fn read_json_schema( + uri: &str, + parse_options: Option, + max_bytes: Option, + io_client: Arc, + io_stats: Option, +) -> DaftResult { + let runtime_handle = get_runtime(true)?; + let _rt_guard = runtime_handle.enter(); + runtime_handle.block_on(async { + read_json_schema_single( + uri, + parse_options.unwrap_or_default(), + // Default to 1 MiB. + max_bytes.or(Some(1024 * 1024)), + io_client, + io_stats, + ) + .await + }) +} + +pub async fn read_json_schema_bulk( + uris: &[&str], + parse_options: Option, + max_bytes: Option, + io_client: Arc, + io_stats: Option, + num_parallel_tasks: usize, +) -> DaftResult> { + let runtime_handle = get_runtime(true)?; + let _rt_guard = runtime_handle.enter(); + let result = runtime_handle + .block_on(async { + let task_stream = futures::stream::iter(uris.iter().map(|uri| { + let owned_string = uri.to_string(); + let owned_client = io_client.clone(); + let owned_io_stats = io_stats.clone(); + let owned_parse_options = parse_options.clone(); + tokio::spawn(async move { + read_json_schema_single( + &owned_string, + owned_parse_options.unwrap_or_default(), + max_bytes, + owned_client, + owned_io_stats, + ) + .await + }) + })); + task_stream + .buffered(num_parallel_tasks) + .try_collect::>() + .await + }) + .context(super::JoinSnafu {})?; + result.into_iter().collect::>>() +} + +pub(crate) async fn read_json_schema_single( + uri: &str, + _: JsonParseOptions, + max_bytes: Option, + io_client: Arc, + io_stats: Option, +) -> DaftResult { + let (reader, max_bytes): (Box, Option) = match io_client + .single_url_get(uri.to_string(), None, io_stats) + .await? + { + GetResult::File(file) => ( + Box::new(BufReader::new(File::open(file.path).await?)), + max_bytes, + ), + GetResult::Stream(stream, size, _) => ( + Box::new(StreamReader::new(stream)), + // Truncate max_bytes to size if both are set. + max_bytes.map(|m| size.map(|s| m.min(s)).unwrap_or(m)), + ), + }; + // If file is compressed, wrap stream in decoding stream. + let reader: Box = match CompressionCodec::from_uri(uri) { + Some(compression) => Box::new(tokio::io::BufReader::new(compression.to_decoder(reader))), + None => reader, + }; + let arrow_schema = infer_schema(reader, None, max_bytes).await?; + let schema = Schema::try_from(&arrow_schema)?; + Ok(schema) +} + +async fn infer_schema( + reader: R, + max_rows: Option, + max_bytes: Option, +) -> DaftResult +where + R: tokio::io::AsyncBufRead + Unpin + Send, +{ + let max_records = max_rows.unwrap_or(usize::MAX); + let max_bytes = max_bytes.unwrap_or(usize::MAX); + let mut total_bytes = 0; + // Stream of unparsed JSON string records. + let line_stream = tokio_stream::wrappers::LinesStream::new(reader.lines()); + let mut schema_stream = line_stream + .try_take_while(|record| { + // Terminate scan if we've exceeded our max_bytes threshold with the last-read line. + if total_bytes >= max_bytes { + futures::future::ready(Ok(false)) + } else { + total_bytes += record.len(); + futures::future::ready(Ok(true)) + } + }) + .take(max_records) + .map(|record| { + let record = record.context(StdIOSnafu)?; + + // Parse record into a JSON Value, then infer the schema. + let parsed_record = + parse(record.as_bytes()).map_err(|e| super::Error::JsonDeserializationError { + string: e.to_string(), + })?; + infer_records_schema(&parsed_record).context(ArrowSnafu) + }); + // Collect all infered dtypes for each column. + let mut column_types: IndexMap> = IndexMap::new(); + while let Some(schema) = schema_stream.next().await.transpose()? { + for field in schema.fields { + // Get-and-mutate-or-insert. + match column_types.entry(field.name) { + indexmap::map::Entry::Occupied(mut v) => { + v.get_mut().insert(field.data_type); + } + indexmap::map::Entry::Vacant(v) => { + let mut a = HashSet::new(); + a.insert(field.data_type); + v.insert(a); + } + } + } + } + // Convert column types map to dtype-consolidated column fields. + let fields = column_types_map_to_fields(column_types); + Ok(fields.into()) +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use common_error::DaftResult; + use daft_core::{ + datatypes::{Field, TimeUnit}, + schema::Schema, + DataType, + }; + use daft_io::{IOClient, IOConfig}; + use rstest::rstest; + + use super::read_json_schema; + + #[rstest] + fn test_json_schema_local( + #[values( + // Uncompressed + None, + // brotli + Some("br"), + // bzip2 + Some("bz2"), + // TODO(Clark): Add deflate compressed JSON file to test data fixtures. + // // deflate + // Some("deflate"), + // gzip + Some("gz"), + // lzma + Some("lzma"), + // xz + Some("xz"), + // zlib + Some("zl"), + // zstd + Some("zst"), + )] + compression: Option<&str>, + ) -> DaftResult<()> { + let file = format!( + "{}/test/iris_tiny.jsonl{}", + env!("CARGO_MANIFEST_DIR"), + compression.map_or("".to_string(), |ext| format!(".{}", ext)) + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = read_json_schema(file.as_ref(), None, None, io_client.clone(), None)?; + assert_eq!( + schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])?, + ); + + Ok(()) + } + + #[rstest] + fn test_json_schema_local_dtypes() -> DaftResult<()> { + let file = format!("{}/test/dtypes.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = read_json_schema(file.as_ref(), None, None, io_client, None)?; + assert_eq!( + schema, + Schema::new(vec![ + Field::new("int", DataType::Int64), + Field::new("float", DataType::Float64), + Field::new("bool", DataType::Boolean), + Field::new("str", DataType::Utf8), + Field::new("null", DataType::Null), + Field::new("date", DataType::Date), + // TODO(Clark): Add coverage for time parsing once we add support for representing time series in Daft. + // // Time unit should be coarest granularity found in file, i.e. seconds. + // Field::new("time", DataType::Time(TimeUnit::Nanoseconds)), + // Time unit should be coarsest granularity found in file, i.e. seconds due to naive date inclusion. + Field::new( + "naive_timestamp", + DataType::Timestamp(TimeUnit::Seconds, None) + ), + // Timezone should be UTC due to field having multiple different timezones across records. + // Time unit should be coarsest granularity found in file, i.e. milliseconds. + Field::new( + "timestamp", + DataType::Timestamp(TimeUnit::Milliseconds, Some("Z".to_string())) + ), + Field::new("list", DataType::List(Box::new(DataType::Int64))), + Field::new( + "obj", + DataType::Struct(vec![ + Field::new("a", DataType::Int64), + Field::new("b", DataType::Boolean) + ]) + ), + Field::new( + "nested_list", + DataType::List(Box::new(DataType::List(Box::new(DataType::Struct(vec![ + Field::new("a", DataType::Utf8), + ]))))) + ), + Field::new( + "nested_obj", + DataType::Struct(vec![ + Field::new( + "obj", + DataType::Struct(vec![Field::new("a", DataType::Int64)]) + ), + Field::new("list", DataType::List(Box::new(DataType::Int64))), + ]) + ), + ])? + .into(), + ); + + Ok(()) + } + + #[test] + fn test_json_schema_local_nulls() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny_nulls.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = read_json_schema(file.as_ref(), None, None, io_client.clone(), None)?; + assert_eq!( + schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])?, + ); + + Ok(()) + } + + #[test] + fn test_json_schema_local_conflicting_types_utf8_fallback() -> DaftResult<()> { + let file = format!( + "{}/test/iris_tiny_conflicting_dtypes.jsonl", + env!("CARGO_MANIFEST_DIR"), + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = read_json_schema(file.as_ref(), None, None, io_client.clone(), None)?; + assert_eq!( + schema, + Schema::new(vec![ + // All conflicting dtypes fall back to UTF8. + Field::new("sepalLength", DataType::Utf8), + Field::new("sepalWidth", DataType::Utf8), + Field::new("petalLength", DataType::Utf8), + // Float + Int => Float, non-conflicting + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])?, + ); + + Ok(()) + } + + #[test] + fn test_json_schema_local_max_bytes() -> DaftResult<()> { + let file = format!("{}/test/iris_tiny.jsonl", env!("CARGO_MANIFEST_DIR"),); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = read_json_schema(file.as_ref(), None, Some(100), io_client.clone(), None)?; + assert_eq!( + schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])?, + ); + + Ok(()) + } + + #[rstest] + fn test_json_schema_s3( + #[values( + // Uncompressed + None, + // brotli + Some("br"), + // bzip2 + Some("bz2"), + // TODO(Clark): Add deflate compressed JSON file to test data fixtures. + // deflate + // Some("deflate"), + // gzip + Some("gz"), + // lzma + Some("lzma"), + // xz + Some("xz"), + // zlib + Some("zl"), + // zstd + Some("zst"), + )] + compression: Option<&str>, + ) -> DaftResult<()> { + let file = format!( + "s3://daft-public-data/test_fixtures/json-dev/iris_tiny.jsonl{}", + compression.map_or("".to_string(), |ext| format!(".{}", ext)) + ); + + let mut io_config = IOConfig::default(); + io_config.s3.anonymous = true; + let io_client = Arc::new(IOClient::new(io_config.into())?); + + let schema = read_json_schema(file.as_ref(), None, None, io_client.clone(), None)?; + assert_eq!( + schema, + Schema::new(vec![ + Field::new("sepalLength", DataType::Float64), + Field::new("sepalWidth", DataType::Float64), + Field::new("petalLength", DataType::Float64), + Field::new("petalWidth", DataType::Float64), + Field::new("species", DataType::Utf8), + ])? + ); + + Ok(()) + } +} diff --git a/src/daft-json/test/dtypes.jsonl b/src/daft-json/test/dtypes.jsonl new file mode 100644 index 0000000000..3986d97356 --- /dev/null +++ b/src/daft-json/test/dtypes.jsonl @@ -0,0 +1,4 @@ +{"int": 1, "float": 2.3, "bool": false, "str": "foo", "null": null, "date": "2023-11-29", "naive_timestamp": "2023-11-29T06:31:52.342", "timestamp": "2023-11-29T06:31:52.342567Z", "list": [1, 2, 3], "obj": {"a": 1, "b": false}, "nested_list": [[{"a": "foo"}]], "nested_obj": {"obj": {"a": 4}, "list": [1, 2]}} +{"int": 2, "float": 3.3, "bool": true, "str": "bar", "null": null, "date": "2023/11/28", "naive_timestamp": "2023-11-29", "timestamp": "2023-11-29T06:31:52.342+07:00", "list": [4, 5], "obj": {"a": 2, "b": false}, "nested_list": [[{"a": "bar"}]], "nested_obj": {"obj": {"a": 6}, "list": [3, 4]}} +{"int": null, "float": null, "bool": null, "str": null, "null": null, "date": null, "naive_timestamp": null, "timestamp": null, "list": null, "obj": null, "nested_list": null, "nested_obj": null} +{"int": 3, "float": 4.3, "bool": false, "str": "baz", "null": null, "date": "2023-11-27", "naive_timestamp": "2023-11-29 06:31:52.342567", "timestamp": "2023-11-29 06:31:52.342-07:00", "list": [6, 7, null, 9], "obj": {"a": null, "b": false}, "nested_list": [[{"a": null}]], "nested_obj": {"obj": {"a": null}, "list": [5, null]}} diff --git a/src/daft-json/test/iris_tiny.jsonl b/src/daft-json/test/iris_tiny.jsonl new file mode 100644 index 0000000000..99926cc27d --- /dev/null +++ b/src/daft-json/test/iris_tiny.jsonl @@ -0,0 +1,20 @@ +{"sepalLength": 5.1, "sepalWidth": 3.5, "petalLength": 1.4, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.9, "sepalWidth": 3.0, "petalLength": 1.4, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.7, "sepalWidth": 3.2, "petalLength": 1.3, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.6, "sepalWidth": 3.1, "petalLength": 1.5, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 5.0, "sepalWidth": 3.6, "petalLength": 1.4, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 5.4, "sepalWidth": 3.9, "petalLength": 1.7, "petalWidth": 0.4, "species": "setosa"} +{"sepalLength": 4.6, "sepalWidth": 3.4, "petalLength": 1.4, "petalWidth": 0.3, "species": "setosa"} +{"sepalLength": 5.0, "sepalWidth": 3.4, "petalLength": 1.5, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.4, "sepalWidth": 2.9, "petalLength": 1.4, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.9, "sepalWidth": 3.1, "petalLength": 1.5, "petalWidth": 0.1, "species": "setosa"} +{"sepalLength": 5.4, "sepalWidth": 3.7, "petalLength": 1.5, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.8, "sepalWidth": 3.4, "petalLength": 1.6, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.8, "sepalWidth": 3.0, "petalLength": 1.4, "petalWidth": 0.1, "species": "setosa"} +{"sepalLength": 4.3, "sepalWidth": 3.0, "petalLength": 1.1, "petalWidth": 0.1, "species": "setosa"} +{"sepalLength": 5.8, "sepalWidth": 4.0, "petalLength": 1.2, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 5.7, "sepalWidth": 4.4, "petalLength": 1.5, "petalWidth": 0.4, "species": "setosa"} +{"sepalLength": 5.4, "sepalWidth": 3.9, "petalLength": 1.3, "petalWidth": 0.4, "species": "setosa"} +{"sepalLength": 5.1, "sepalWidth": 3.5, "petalLength": 1.4, "petalWidth": 0.3, "species": "setosa"} +{"sepalLength": 5.7, "sepalWidth": 3.8, "petalLength": 1.7, "petalWidth": 0.3, "species": "setosa"} +{"sepalLength": 5.1, "sepalWidth": 3.8, "petalLength": 1.5, "petalWidth": 0.3, "species": "setosa"} diff --git a/src/daft-json/test/iris_tiny.jsonl.br b/src/daft-json/test/iris_tiny.jsonl.br new file mode 100644 index 0000000000..9bf69cb456 Binary files /dev/null and b/src/daft-json/test/iris_tiny.jsonl.br differ diff --git a/src/daft-json/test/iris_tiny.jsonl.bz2 b/src/daft-json/test/iris_tiny.jsonl.bz2 new file mode 100644 index 0000000000..1fa7f678d7 Binary files /dev/null and b/src/daft-json/test/iris_tiny.jsonl.bz2 differ diff --git a/src/daft-json/test/iris_tiny.jsonl.gz b/src/daft-json/test/iris_tiny.jsonl.gz new file mode 100644 index 0000000000..ada0d0117e Binary files /dev/null and b/src/daft-json/test/iris_tiny.jsonl.gz differ diff --git a/src/daft-json/test/iris_tiny.jsonl.lzma b/src/daft-json/test/iris_tiny.jsonl.lzma new file mode 100644 index 0000000000..33b43e5c79 Binary files /dev/null and b/src/daft-json/test/iris_tiny.jsonl.lzma differ diff --git a/src/daft-json/test/iris_tiny.jsonl.xz b/src/daft-json/test/iris_tiny.jsonl.xz new file mode 100644 index 0000000000..43a5951106 Binary files /dev/null and b/src/daft-json/test/iris_tiny.jsonl.xz differ diff --git a/src/daft-json/test/iris_tiny.jsonl.zl b/src/daft-json/test/iris_tiny.jsonl.zl new file mode 100644 index 0000000000..0c4c91f92d Binary files /dev/null and b/src/daft-json/test/iris_tiny.jsonl.zl differ diff --git a/src/daft-json/test/iris_tiny.jsonl.zst b/src/daft-json/test/iris_tiny.jsonl.zst new file mode 100644 index 0000000000..2ecfeb5fde Binary files /dev/null and b/src/daft-json/test/iris_tiny.jsonl.zst differ diff --git a/src/daft-json/test/iris_tiny_all_null_column.jsonl b/src/daft-json/test/iris_tiny_all_null_column.jsonl new file mode 100644 index 0000000000..3e73b3ef6a --- /dev/null +++ b/src/daft-json/test/iris_tiny_all_null_column.jsonl @@ -0,0 +1,6 @@ +{"sepalLength": 5.1, "sepalWidth": 3.5, "petalLength": null, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.9, "sepalWidth": 3.0, "petalLength": null, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.7, "sepalWidth": 3.2, "petalLength": null, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.6, "sepalWidth": 3.1, "petalLength": null, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 5.0, "sepalWidth": 3.6, "petalLength": null, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 5.4, "sepalWidth": 3.9, "petalLength": null, "petalWidth": 0.4, "species": "setosa"} diff --git a/src/daft-json/test/iris_tiny_conflicting_dtypes.jsonl b/src/daft-json/test/iris_tiny_conflicting_dtypes.jsonl new file mode 100644 index 0000000000..351361be62 --- /dev/null +++ b/src/daft-json/test/iris_tiny_conflicting_dtypes.jsonl @@ -0,0 +1,2 @@ +{"sepalLength": 5.1, "sepalWidth": false, "petalLength": 3, "petalWidth": 3, "species": "setosa"} +{"sepalLength": "foo", "sepalWidth": 3.0, "petalLength": "bar", "petalWidth": 0.2, "species": false} diff --git a/src/daft-json/test/iris_tiny_nulls.jsonl b/src/daft-json/test/iris_tiny_nulls.jsonl new file mode 100644 index 0000000000..71dd3eb231 --- /dev/null +++ b/src/daft-json/test/iris_tiny_nulls.jsonl @@ -0,0 +1,6 @@ +{"sepalLength": null, "sepalWidth": 3.5, "petalLength": 1.4, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.9, "sepalWidth": null, "petalLength": 1.4, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.7, "sepalWidth": 3.2, "petalLength": null, "petalWidth": 0.2, "species": "setosa"} +{"sepalLength": 4.6, "sepalWidth": 3.1, "petalLength": 1.5, "petalWidth": null, "species": "setosa"} +{"sepalLength": 5.0, "sepalWidth": 3.6, "petalLength": 1.4, "petalWidth": 0.2, "species": null} +{"sepalLength": null, "sepalWidth": null, "petalLength": null, "petalWidth": null, "species": null} diff --git a/src/daft-micropartition/Cargo.toml b/src/daft-micropartition/Cargo.toml index f74c46a53c..5c7e026dc2 100644 --- a/src/daft-micropartition/Cargo.toml +++ b/src/daft-micropartition/Cargo.toml @@ -6,6 +6,7 @@ daft-core = {path = "../daft-core", default-features = false} daft-csv = {path = "../daft-csv", default-features = false} daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} +daft-json = {path = "../daft-json", default-features = false} daft-parquet = {path = "../daft-parquet", default-features = false} daft-scan = {path = "../daft-scan", default-features = false} daft-stats = {path = "../daft-stats", default-features = false} diff --git a/src/daft-micropartition/src/micropartition.rs b/src/daft-micropartition/src/micropartition.rs index b272023d46..c662bcd6f7 100644 --- a/src/daft-micropartition/src/micropartition.rs +++ b/src/daft-micropartition/src/micropartition.rs @@ -8,6 +8,7 @@ use common_error::DaftResult; use daft_core::schema::{Schema, SchemaRef}; use daft_csv::{CsvConvertOptions, CsvParseOptions, CsvReadOptions}; +use daft_json::{JsonConvertOptions, JsonParseOptions, JsonReadOptions}; use daft_parquet::read::{ read_parquet_bulk, read_parquet_metadata_bulk, ParquetSchemaInferenceOptions, }; @@ -131,7 +132,7 @@ fn materialize_scan_task( // **************** // Native CSV Reads // **************** - FileFormatConfig::Csv(cfg @ CsvSourceConfig { .. }) => { + FileFormatConfig::Csv(cfg) => { let col_names = if !cfg.has_headers { Some( cast_to_schema @@ -182,8 +183,30 @@ fn materialize_scan_task( // **************** // Native JSON Reads // **************** - FileFormatConfig::Json(_) => { - todo!("TODO: Implement MicroPartition native reads for JSON."); + FileFormatConfig::Json(cfg) => { + let convert_options = JsonConvertOptions::new_internal( + scan_task.pushdowns.limit, + column_names + .as_ref() + .map(|cols| cols.iter().map(|col| col.to_string()).collect()), + None, + ); + let parse_options = JsonParseOptions::new_internal(); + let read_options = + JsonReadOptions::new_internal(cfg.buffer_size, cfg.chunk_size); + let uris = urls.collect::>(); + daft_json::read_json_bulk( + uris.as_slice(), + Some(convert_options), + Some(parse_options), + Some(read_options), + io_client, + io_stats, + native_storage_config.multithreaded_io, + None, + 8, + ) + .context(DaftCoreComputeSnafu)? } } } @@ -628,6 +651,55 @@ pub(crate) fn read_csv_into_micropartition( } } +pub(crate) fn read_json_into_micropartition( + uris: &[&str], + convert_options: Option, + parse_options: Option, + read_options: Option, + io_config: Arc, + multithreaded_io: bool, + io_stats: Option, +) -> DaftResult { + let io_client = daft_io::get_io_client(multithreaded_io, io_config.clone())?; + + match uris { + [] => Ok(MicroPartition::empty(None)), + uris => { + // Perform a bulk read of URIs, materializing a table per URI. + let tables = daft_json::read_json_bulk( + uris, + convert_options, + parse_options, + read_options, + io_client, + io_stats, + multithreaded_io, + None, + 8, + ) + .context(DaftCoreComputeSnafu)?; + + // Union all schemas and cast all tables to the same schema + let unioned_schema = tables + .iter() + .map(|tbl| tbl.schema.clone()) + .try_reduce(|s1, s2| s1.union(s2.as_ref()).map(Arc::new))? + .unwrap(); + let tables = tables + .into_iter() + .map(|tbl| tbl.cast_to_schema(&unioned_schema)) + .collect::>>()?; + + // Construct MicroPartition from tables and unioned schema + Ok(MicroPartition::new_loaded( + unioned_schema.clone(), + Arc::new(tables), + None, + )) + } + } +} + #[allow(clippy::too_many_arguments)] pub(crate) fn read_parquet_into_micropartition( uris: &[&str], diff --git a/src/daft-micropartition/src/python.rs b/src/daft-micropartition/src/python.rs index f2a6796ffc..7e398107c2 100644 --- a/src/daft-micropartition/src/python.rs +++ b/src/daft-micropartition/src/python.rs @@ -12,6 +12,7 @@ use daft_core::{ use daft_csv::{CsvConvertOptions, CsvParseOptions, CsvReadOptions}; use daft_dsl::python::PyExpr; use daft_io::{python::IOConfig, IOStatsContext}; +use daft_json::{JsonConvertOptions, JsonParseOptions, JsonReadOptions}; use daft_parquet::read::ParquetSchemaInferenceOptions; use daft_scan::{python::pylib::PyScanTask, storage_config::PyStorageConfig, ScanTask}; use daft_stats::TableStatistics; @@ -370,6 +371,33 @@ impl PyMicroPartition { Ok(mp.into()) } + #[staticmethod] + pub fn read_json_native( + py: Python, + uri: &str, + convert_options: Option, + parse_options: Option, + read_options: Option, + io_config: Option, + multithreaded_io: Option, + ) -> PyResult { + let mp = py.allow_threads(|| { + let io_stats = IOStatsContext::new(format!("read_json: for uri {uri}")); + let io_config = io_config.unwrap_or_default().config.into(); + + crate::micropartition::read_json_into_micropartition( + [uri].as_ref(), + convert_options, + parse_options, + read_options, + io_config, + multithreaded_io.unwrap_or(true), + Some(io_stats), + ) + })?; + Ok(mp.into()) + } + #[staticmethod] pub fn read_csv( py: Python, diff --git a/src/daft-plan/src/optimization/optimizer.rs b/src/daft-plan/src/optimization/optimizer.rs index d020c0eb99..72670e77b1 100644 --- a/src/daft-plan/src/optimization/optimizer.rs +++ b/src/daft-plan/src/optimization/optimizer.rs @@ -515,14 +515,16 @@ mod tests { ], OptimizerConfig::new(20), ); + let fields = vec![Field::new("a", DataType::Int64)]; let proj_exprs = vec![ col("a") + lit(1), (col("a") + lit(2)).alias("b"), (col("a") + lit(3)).alias("c"), ]; - let plan = dummy_scan_node(vec![Field::new("a", DataType::Int64)]) + let filter_predicate = col("a").lt(&lit(2)); + let plan = dummy_scan_node(fields.clone()) .project(proj_exprs, Default::default())? - .filter(col("a").lt(&lit(2)))? + .filter(filter_predicate)? .build(); let mut pass_count = 0; let mut did_transform = false; @@ -536,7 +538,7 @@ mod tests { let expected = "\ Filter: [[[col(a) < lit(2)] | lit(false)] | lit(false)] & lit(true)\ \n Project: col(a) + lit(3) AS c, col(a) + lit(1), col(a) + lit(2) AS b\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64)"; assert_eq!(opt_plan.repr_indent(), expected); Ok(()) } diff --git a/src/daft-plan/src/optimization/rules/drop_repartition.rs b/src/daft-plan/src/optimization/rules/drop_repartition.rs index 02db0ffb1f..0593151c07 100644 --- a/src/daft-plan/src/optimization/rules/drop_repartition.rs +++ b/src/daft-plan/src/optimization/rules/drop_repartition.rs @@ -98,7 +98,7 @@ mod tests { .build(); let expected = "\ Repartition: Scheme = Hash, Number of partitions = 5, Partition by = col(a)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } diff --git a/src/daft-plan/src/optimization/rules/push_down_filter.rs b/src/daft-plan/src/optimization/rules/push_down_filter.rs index 0bd738e574..dbe7c72102 100644 --- a/src/daft-plan/src/optimization/rules/push_down_filter.rs +++ b/src/daft-plan/src/optimization/rules/push_down_filter.rs @@ -261,7 +261,7 @@ mod tests { .build(); let expected = "\ Filter: [col(b) == lit(\"foo\")] & [col(a) < lit(2)]\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -279,7 +279,7 @@ mod tests { let expected = "\ Project: col(a)\ \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -297,7 +297,7 @@ mod tests { let expected = "\ Project: col(a), col(b)\ \n Filter: [col(a) < lit(2)] & [col(b) == lit(\"foo\")]\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -317,7 +317,7 @@ mod tests { let expected = "\ Filter: col(a) < lit(2)\ \n Project: col(a) + lit(1)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -338,7 +338,7 @@ mod tests { let expected = "\ Project: col(a) + lit(1)\ \n Filter: [col(a) + lit(1)] < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -356,7 +356,7 @@ mod tests { let expected = "\ Sort: Sort by = (col(a), descending)\ \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; // TODO(Clark): For tests in which we only care about reordering of operators, maybe switch to a form that leverages the single-node display? // let expected = format!("{sort}\n {filter}\n {source}"); assert_optimized_plan_eq(plan, expected)?; @@ -376,7 +376,7 @@ mod tests { let expected = "\ Repartition: Scheme = Hash, Number of partitions = 1, Partition by = col(a)\ \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -395,9 +395,9 @@ mod tests { let expected = "\ Concat\ \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)\ + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)\ \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -423,8 +423,8 @@ mod tests { let expected = "\ Join: Type = Inner, On = col(b), Output schema = a (Int64), b (Utf8), c (Float64)\ \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)\ - \n Source: Json, File paths = [/foo], File schema = b (Utf8), c (Float64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = b (Utf8), c (Float64)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)\ + \n Source: Json, File paths = [/foo], File schema = b (Utf8), c (Float64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = b (Utf8), c (Float64)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -449,9 +449,9 @@ mod tests { .build(); let expected = "\ Join: Type = Inner, On = col(b), Output schema = a (Int64), b (Utf8), c (Float64)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)\ + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)\ \n Filter: col(c) < lit(2.0)\ - \n Source: Json, File paths = [/foo], File schema = b (Utf8), c (Float64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = b (Utf8), c (Float64)"; + \n Source: Json, File paths = [/foo], File schema = b (Utf8), c (Float64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = b (Utf8), c (Float64)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -475,9 +475,9 @@ mod tests { let expected = "\ Join: Type = Inner, On = col(b), Output schema = a (Int64), b (Int64), c (Float64)\ \n Filter: col(b) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), c (Float64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64), c (Float64)\ + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), c (Float64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64), c (Float64)\ \n Filter: col(b) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = b (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = b (Int64)"; + \n Source: Json, File paths = [/foo], File schema = b (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = b (Int64)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } diff --git a/src/daft-plan/src/optimization/rules/push_down_limit.rs b/src/daft-plan/src/optimization/rules/push_down_limit.rs index 457dcbe830..a9706db4f1 100644 --- a/src/daft-plan/src/optimization/rules/push_down_limit.rs +++ b/src/daft-plan/src/optimization/rules/push_down_limit.rs @@ -87,6 +87,7 @@ mod tests { use common_error::DaftResult; use daft_core::{datatypes::Field, schema::Schema, DataType}; use daft_dsl::col; + use daft_scan::Pushdowns; use std::sync::Arc; #[cfg(feature = "python")] @@ -98,7 +99,7 @@ mod tests { rules::PushDownLimit, Optimizer, }, - test::{dummy_scan_node, dummy_scan_node_with_limit}, + test::{dummy_scan_node, dummy_scan_node_with_pushdowns}, LogicalPlan, PartitionScheme, }; @@ -139,7 +140,7 @@ mod tests { .build(); let expected = "\ Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -149,18 +150,18 @@ mod tests { /// Limit-Source[existing_limit] -> Source[existing_limit] #[test] fn limit_does_not_push_into_external_source_if_smaller_limit() -> DaftResult<()> { - let plan = dummy_scan_node_with_limit( + let plan = dummy_scan_node_with_pushdowns( vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), ], - Some(3), + Pushdowns::default().with_limit(Some(3)), ) .limit(5, false)? .build(); let expected = "\ Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 3, Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 3, Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -170,18 +171,18 @@ mod tests { /// Limit-Source[existing_limit] -> Source[new_limit] #[test] fn limit_does_push_into_external_source_if_larger_limit() -> DaftResult<()> { - let plan = dummy_scan_node_with_limit( + let plan = dummy_scan_node_with_pushdowns( vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), ], - Some(10), + Pushdowns::default().with_limit(Some(10)), ) .limit(5, false)? .build(); let expected = "\ Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -199,7 +200,7 @@ mod tests { .build(); let expected = "\ Limit: 5\ - \n . Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; + \n . Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -219,7 +220,7 @@ mod tests { let expected = "\ Repartition: Scheme = Hash, Number of partitions = 1, Partition by = col(a)\ \n Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -239,7 +240,7 @@ mod tests { let expected = "\ Project: col(a)\ \n Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Utf8), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Limit pushdown = 5, Output schema = a (Int64), b (Utf8)"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } diff --git a/src/daft-plan/src/optimization/rules/push_down_projection.rs b/src/daft-plan/src/optimization/rules/push_down_projection.rs index edd04dab16..b8af4b559f 100644 --- a/src/daft-plan/src/optimization/rules/push_down_projection.rs +++ b/src/daft-plan/src/optimization/rules/push_down_projection.rs @@ -573,7 +573,7 @@ mod tests { let expected = "\ Project: [col(a) + lit(1)] + lit(3), col(b) + lit(2), col(a) + lit(4)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; assert_optimized_plan_eq(unoptimized, expected)?; Ok(()) } @@ -589,7 +589,7 @@ mod tests { .build(); let expected = "\ - Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; + Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; assert_optimized_plan_eq(unoptimized, expected)?; Ok(()) @@ -606,7 +606,7 @@ mod tests { let expected = "\ Project: col(b), col(a)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; assert_optimized_plan_eq(unoptimized, expected)?; Ok(()) @@ -624,7 +624,7 @@ mod tests { let expected = "\ Project: col(b) + lit(3)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Projection pushdown = [b], Output schema = b (Int64)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Projection pushdown = [b], Output schema = b (Int64)"; assert_optimized_plan_eq(unoptimized, expected)?; Ok(()) @@ -650,7 +650,7 @@ mod tests { let expected = "\ Project: col(a), col(b), col(b) AS c\ \n Project: col(b) + lit(3), col(a)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Output schema = a (Int64), b (Int64)"; assert_optimized_plan_eq(unoptimized, expected)?; Ok(()) @@ -671,7 +671,7 @@ mod tests { let expected = "\ Project: col(a)\ \n Aggregation: mean(col(a)), Group by = col(c), Output schema = c (Int64), a (Float64)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), c (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Projection pushdown = [a, c], Output schema = a (Int64), c (Int64)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Int64), c (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Projection pushdown = [a, c], Output schema = a (Int64), c (Int64)"; assert_optimized_plan_eq(unoptimized, expected)?; Ok(()) @@ -692,7 +692,7 @@ mod tests { let expected = "\ Project: col(a)\ \n Filter: col(b)\ - \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Boolean), c (Int64), Format-specific config = Json(JsonSourceConfig), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Projection pushdown = [a, b], Output schema = a (Int64), b (Boolean)"; + \n Source: Json, File paths = [/foo], File schema = a (Int64), b (Boolean), c (Int64), Format-specific config = Json(JsonSourceConfig { buffer_size: None, chunk_size: None }), Storage config = Native(NativeStorageConfig { io_config: None, multithreaded_io: true }), Projection pushdown = [a, b], Output schema = a (Int64), b (Boolean)"; assert_optimized_plan_eq(unoptimized, expected)?; Ok(()) diff --git a/src/daft-plan/src/test/mod.rs b/src/daft-plan/src/test/mod.rs index 11b69182df..1039f84ea1 100644 --- a/src/daft-plan/src/test/mod.rs +++ b/src/daft-plan/src/test/mod.rs @@ -3,31 +3,25 @@ use std::sync::Arc; use daft_core::{datatypes::Field, schema::Schema}; use daft_scan::{file_format::FileFormatConfig, storage_config::StorageConfig, Pushdowns}; -use crate::{ - builder::LogicalPlanBuilder, source_info::FileInfos, JsonSourceConfig, NativeStorageConfig, -}; +use crate::{builder::LogicalPlanBuilder, source_info::FileInfos, NativeStorageConfig}; /// Create a dummy scan node containing the provided fields in its schema. pub fn dummy_scan_node(fields: Vec) -> LogicalPlanBuilder { - let schema = Arc::new(Schema::new(fields).unwrap()); - LogicalPlanBuilder::table_scan( - FileInfos::new_internal(vec!["/foo".to_string()], vec![None], vec![None]), - schema, - FileFormatConfig::Json(JsonSourceConfig {}).into(), - StorageConfig::Native(NativeStorageConfig::new_internal(true, None).into()).into(), - ) - .unwrap() + dummy_scan_node_with_pushdowns(fields, Default::default()) } /// Create a dummy scan node containing the provided fields in its schema and the provided limit. -pub fn dummy_scan_node_with_limit(fields: Vec, limit: Option) -> LogicalPlanBuilder { +pub fn dummy_scan_node_with_pushdowns( + fields: Vec, + pushdowns: Pushdowns, +) -> LogicalPlanBuilder { let schema = Arc::new(Schema::new(fields).unwrap()); LogicalPlanBuilder::table_scan_with_pushdowns( FileInfos::new_internal(vec!["/foo".to_string()], vec![None], vec![None]), schema, - FileFormatConfig::Json(JsonSourceConfig {}).into(), + FileFormatConfig::Json(Default::default()).into(), StorageConfig::Native(NativeStorageConfig::new_internal(true, None).into()).into(), - Pushdowns::new(None, None, limit), + pushdowns, ) .unwrap() } diff --git a/src/daft-scan/Cargo.toml b/src/daft-scan/Cargo.toml index 91d6b8b6a4..c46dc61d9f 100644 --- a/src/daft-scan/Cargo.toml +++ b/src/daft-scan/Cargo.toml @@ -6,6 +6,7 @@ daft-core = {path = "../daft-core", default-features = false} daft-csv = {path = "../daft-csv", default-features = false} daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} +daft-json = {path = "../daft-json", default-features = false} daft-parquet = {path = "../daft-parquet", default-features = false} daft-stats = {path = "../daft-stats", default-features = false} daft-table = {path = "../daft-table", default-features = false} diff --git a/src/daft-scan/src/file_format.rs b/src/daft-scan/src/file_format.rs index 5a733bbf7f..c8e63e047a 100644 --- a/src/daft-scan/src/file_format.rs +++ b/src/daft-scan/src/file_format.rs @@ -157,15 +157,38 @@ impl_bincode_py_state_serialization!(CsvSourceConfig); /// Configuration for a JSON data source. #[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize, Hash)] #[cfg_attr(feature = "python", pyclass(module = "daft.daft", get_all))] -pub struct JsonSourceConfig {} +pub struct JsonSourceConfig { + pub buffer_size: Option, + pub chunk_size: Option, +} + +impl JsonSourceConfig { + pub fn new_internal(buffer_size: Option, chunk_size: Option) -> Self { + Self { + buffer_size, + chunk_size, + } + } +} + +impl Default for JsonSourceConfig { + fn default() -> Self { + Self::new_internal(None, None) + } +} #[cfg(feature = "python")] #[pymethods] impl JsonSourceConfig { /// Create a config for a JSON data source. + /// + /// # Arguments + /// + /// * `buffer_size` - Size of the buffer (in bytes) used by the streaming reader. + /// * `chunk_size` - Size of the chunks (in bytes) deserialized in parallel by the streaming reader. #[new] - fn new() -> Self { - Self {} + fn new(buffer_size: Option, chunk_size: Option) -> Self { + Self::new_internal(buffer_size, chunk_size) } } diff --git a/src/daft-scan/src/glob.rs b/src/daft-scan/src/glob.rs index 29e9224917..0175ef003c 100644 --- a/src/daft-scan/src/glob.rs +++ b/src/daft-scan/src/glob.rs @@ -8,12 +8,10 @@ use daft_io::{ }; use daft_parquet::read::ParquetSchemaInferenceOptions; use futures::{stream::BoxStream, StreamExt}; -use snafu::{ResultExt, Snafu}; -#[cfg(feature = "python")] -use {crate::PyIOSnafu, daft_core::schema::Schema, pyo3::Python}; +use snafu::Snafu; use crate::{ - file_format::{CsvSourceConfig, FileFormatConfig, JsonSourceConfig, ParquetSourceConfig}, + file_format::{CsvSourceConfig, FileFormatConfig, ParquetSourceConfig}, storage_config::StorageConfig, DataFileSource, PartitionField, Pushdowns, ScanOperator, ScanTask, ScanTaskRef, }; @@ -192,25 +190,13 @@ impl GlobScanOperator { )?; schema } - FileFormatConfig::Json(JsonSourceConfig {}) => { - // NOTE: Native JSON reads not yet implemented, so we have to delegate to Python here or implement - // a daft_json crate that gives us native JSON schema inference - match storage_config.as_ref() { - StorageConfig::Native(_) => { - todo!("Implement native JSON schema inference in a daft_json crate.") - } - #[cfg(feature = "python")] - StorageConfig::Python(_) => Python::with_gil(|py| { - crate::python::pylib::read_json_schema( - py, - first_filepath.as_str(), - storage_config.clone().into(), - ) - .and_then(|s| Ok(Schema::new(s.schema.fields.values().cloned().collect())?)) - .context(PyIOSnafu) - })?, - } - } + FileFormatConfig::Json(_) => daft_json::schema::read_json_schema( + first_filepath.as_str(), + None, + None, + io_client, + Some(io_stats), + )?, }; let schema = match schema_hint { diff --git a/src/daft-scan/src/python.rs b/src/daft-scan/src/python.rs index 05eb883a1f..a9b945f332 100644 --- a/src/daft-scan/src/python.rs +++ b/src/daft-scan/src/python.rs @@ -315,18 +315,6 @@ partitioning_keys:\n", impl_bincode_py_state_serialization!(PyScanTask); - pub(crate) fn read_json_schema( - py: Python, - uri: &str, - storage_config: PyStorageConfig, - ) -> PyResult { - py.import(pyo3::intern!(py, "daft.table.schema_inference"))? - .getattr(pyo3::intern!(py, "from_json"))? - .call1((uri, storage_config))? - .getattr(pyo3::intern!(py, "_schema"))? - .extract() - } - #[pyclass(module = "daft.daft", name = "PartitionField", frozen)] #[derive(Debug, Clone, Serialize, Deserialize)] pub struct PyPartitionField(Arc); diff --git a/src/lib.rs b/src/lib.rs index fce2431e7c..74aaec2393 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -56,6 +56,7 @@ pub mod pylib { daft_io::register_modules(_py, m)?; daft_parquet::register_modules(_py, m)?; daft_csv::register_modules(_py, m)?; + daft_json::register_modules(_py, m)?; daft_plan::register_modules(_py, m)?; daft_micropartition::register_modules(_py, m)?; daft_scan::register_modules(_py, m)?; diff --git a/tests/dataframe/test_creation.py b/tests/dataframe/test_creation.py index 14add620e4..a3542c9bdd 100644 --- a/tests/dataframe/test_creation.py +++ b/tests/dataframe/test_creation.py @@ -610,7 +610,8 @@ def test_create_dataframe_csv_schema_hints_ignore_random_hint( ### -def test_create_dataframe_json(valid_data: list[dict[str, float]]) -> None: +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_create_dataframe_json(valid_data: list[dict[str, float]], use_native_downloader) -> None: with create_temp_filename() as fname: with open(fname, "w") as f: for data in valid_data: @@ -618,7 +619,7 @@ def test_create_dataframe_json(valid_data: list[dict[str, float]]) -> None: f.write("\n") f.flush() - df = daft.read_json(fname) + df = daft.read_json(fname, use_native_downloader=use_native_downloader) assert df.column_names == COL_NAMES pd_df = df.to_pandas() @@ -626,7 +627,8 @@ def test_create_dataframe_json(valid_data: list[dict[str, float]]) -> None: assert len(pd_df) == len(valid_data) -def test_create_dataframe_multiple_jsons(valid_data: list[dict[str, float]]) -> None: +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_create_dataframe_multiple_jsons(valid_data: list[dict[str, float]], use_native_downloader) -> None: with create_temp_filename() as f1name, create_temp_filename() as f2name: with open(f1name, "w") as f1, open(f2name, "w") as f2: for f in (f1, f2): @@ -635,7 +637,7 @@ def test_create_dataframe_multiple_jsons(valid_data: list[dict[str, float]]) -> f.write("\n") f.flush() - df = daft.read_json([f1name, f2name]) + df = daft.read_json([f1name, f2name], use_native_downloader=use_native_downloader) assert df.column_names == COL_NAMES pd_df = df.to_pandas() @@ -643,7 +645,8 @@ def test_create_dataframe_multiple_jsons(valid_data: list[dict[str, float]]) -> assert len(pd_df) == (len(valid_data) * 2) -def test_create_dataframe_json_column_projection(valid_data: list[dict[str, float]]) -> None: +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_create_dataframe_json_column_projection(valid_data: list[dict[str, float]], use_native_downloader) -> None: with create_temp_filename() as fname: with open(fname, "w") as f: for data in valid_data: @@ -653,7 +656,7 @@ def test_create_dataframe_json_column_projection(valid_data: list[dict[str, floa col_subset = COL_NAMES[:3] - df = daft.read_json(fname) + df = daft.read_json(fname, use_native_downloader=use_native_downloader) df = df.select(*col_subset) assert df.column_names == col_subset @@ -662,14 +665,21 @@ def test_create_dataframe_json_column_projection(valid_data: list[dict[str, floa assert len(pd_df) == len(valid_data) +# TODO(Clark): Debug why this segfaults for the native downloader and is slow for the Python downloader. +# @pytest.mark.parametrize("use_native_downloader", [True, False]) +@pytest.mark.skip def test_create_dataframe_json_https() -> None: - df = daft.read_json("https://github.com/Eventual-Inc/mnist-json/raw/master/mnist_handwritten_test.json.gz") + df = daft.read_json( + "https://github.com/Eventual-Inc/mnist-json/raw/master/mnist_handwritten_test.json.gz", + # use_native_downloader=use_native_downloader, + ) df.collect() assert set(df.column_names) == {"label", "image"} assert len(df) == 10000 -def test_create_dataframe_json_specify_schema(valid_data: list[dict[str, float]]) -> None: +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_create_dataframe_json_specify_schema(valid_data: list[dict[str, float]], use_native_downloader) -> None: with create_temp_filename() as fname: with open(fname, "w") as f: for data in valid_data: @@ -686,6 +696,7 @@ def test_create_dataframe_json_specify_schema(valid_data: list[dict[str, float]] "petal_width": DataType.float32(), "variety": DataType.string(), }, + use_native_downloader=use_native_downloader, ) assert df.column_names == COL_NAMES diff --git a/tests/table/table_io/test_json.py b/tests/table/table_io/test_json.py index 74989eae1f..b6bdf88608 100644 --- a/tests/table/table_io/test_json.py +++ b/tests/table/table_io/test_json.py @@ -1,20 +1,31 @@ from __future__ import annotations -import io +import contextlib import json +import os import pathlib +import tempfile from typing import Any import pytest import daft +from daft.daft import NativeStorageConfig, PythonStorageConfig, StorageConfig from daft.datatype import DataType from daft.logical.schema import Schema from daft.runners.partitioning import TableReadOptions from daft.table import MicroPartition, schema_inference, table_io -def test_read_input(tmpdir): +def storage_config_from_use_native_downloader(use_native_downloader: bool) -> StorageConfig: + if use_native_downloader: + return StorageConfig.native(NativeStorageConfig(True, None)) + else: + return StorageConfig.python(PythonStorageConfig(None)) + + +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_read_input(tmpdir, use_native_downloader): tmpdir = pathlib.Path(tmpdir) data = {"foo": [1, 2, 3]} with open(tmpdir / "file.json", "w") as f: @@ -23,27 +34,32 @@ def test_read_input(tmpdir): f.write("\n") schema = Schema._from_field_name_and_types([("foo", DataType.int64())]) + storage_config = storage_config_from_use_native_downloader(use_native_downloader) # Test pathlib, str and IO - assert table_io.read_json(tmpdir / "file.json", schema=schema).to_pydict() == data - assert table_io.read_json(str(tmpdir / "file.json"), schema=schema).to_pydict() == data + assert table_io.read_json(tmpdir / "file.json", schema=schema, storage_config=storage_config).to_pydict() == data + assert ( + table_io.read_json(str(tmpdir / "file.json"), schema=schema, storage_config=storage_config).to_pydict() == data + ) with open(tmpdir / "file.json", "rb") as f: - assert table_io.read_json(f, schema=schema).to_pydict() == data + if use_native_downloader: + f = tmpdir / "file.json" + assert table_io.read_json(f, schema=schema, storage_config=storage_config).to_pydict() == data +@contextlib.contextmanager def _json_write_helper(data: dict[str, list[Any]]): - first_key = list(data.keys())[0] - data_len = len(data[first_key]) - for k in data: - assert len(data[k]) == data_len - - f = io.StringIO() - for i in range(data_len): - json.dump({k: data[k][i] for k in data}, f) - f.write("\n") - f.seek(0) - return io.BytesIO(f.getvalue().encode("utf-8")) + with tempfile.TemporaryDirectory() as directory_name: + first_key = list(data.keys())[0] + data_len = len(data[first_key]) + for k in data: + assert len(data[k]) == data_len + file = os.path.join(directory_name, "tempfile") + with open(file, "w", newline="") as f: + for i in range(data_len): + f.write(json.dumps({k: data[k][i] for k in data}) + "\n") + yield file @pytest.mark.parametrize( @@ -62,16 +78,17 @@ def _json_write_helper(data: dict[str, list[Any]]): ([1, None, 2], DataType.list(DataType.int64())), ], ) -def test_json_infer_schema(data, expected_dtype): - f = _json_write_helper( +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_json_infer_schema(data, expected_dtype, use_native_downloader): + with _json_write_helper( { "id": [1, 2, 3], "data": [data, data, None], } - ) - - schema = schema_inference.from_json(f) - assert schema == Schema._from_field_name_and_types([("id", DataType.int64()), ("data", expected_dtype)]) + ) as f: + storage_config = storage_config_from_use_native_downloader(use_native_downloader) + schema = schema_inference.from_json(f, storage_config=storage_config) + assert schema == Schema._from_field_name_and_types([("id", DataType.int64()), ("data", expected_dtype)]) @pytest.mark.parametrize( @@ -85,57 +102,64 @@ def test_json_infer_schema(data, expected_dtype): ({"foo": 1}, daft.Series.from_pylist([{"foo": 1}, {"foo": 1}, None])), ], ) -def test_json_read_data(data, expected_data_series): - f = _json_write_helper( +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_json_read_data(data, expected_data_series, use_native_downloader): + with _json_write_helper( { "id": [1, 2, 3], "data": [data, data, None], } - ) - - schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", expected_data_series.datatype())]) - expected = MicroPartition.from_pydict( - { - "id": [1, 2, 3], - "data": expected_data_series, - } - ) - table = table_io.read_json(f, schema) - assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" - - -def test_json_read_data_limit_rows(): - f = _json_write_helper( + ) as f: + schema = Schema._from_field_name_and_types( + [("id", DataType.int64()), ("data", expected_data_series.datatype())] + ) + expected = MicroPartition.from_pydict( + { + "id": [1, 2, 3], + "data": expected_data_series, + } + ) + storage_config = storage_config_from_use_native_downloader(use_native_downloader) + table = table_io.read_json(f, schema, storage_config=storage_config) + assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" + + +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_json_read_data_limit_rows(use_native_downloader): + with _json_write_helper( { "id": [1, 2, 3], "data": [1, 2, None], } - ) - - schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = MicroPartition.from_pydict( - { - "id": [1, 2], - "data": [1, 2], - } - ) - table = table_io.read_json(f, schema, read_options=TableReadOptions(num_rows=2)) - assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" - - -def test_json_read_data_select_columns(): - f = _json_write_helper( + ) as f: + schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) + expected = MicroPartition.from_pydict( + { + "id": [1, 2], + "data": [1, 2], + } + ) + storage_config = storage_config_from_use_native_downloader(use_native_downloader) + table = table_io.read_json(f, schema, read_options=TableReadOptions(num_rows=2), storage_config=storage_config) + assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" + + +@pytest.mark.parametrize("use_native_downloader", [True, False]) +def test_json_read_data_select_columns(use_native_downloader): + with _json_write_helper( { "id": [1, 2, 3], "data": [1, 2, None], } - ) - - schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = MicroPartition.from_pydict( - { - "data": [1, 2, None], - } - ) - table = table_io.read_json(f, schema, read_options=TableReadOptions(column_names=["data"])) - assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" + ) as f: + schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) + expected = MicroPartition.from_pydict( + { + "data": [1, 2, None], + } + ) + storage_config = storage_config_from_use_native_downloader(use_native_downloader) + table = table_io.read_json( + f, schema, read_options=TableReadOptions(column_names=["data"]), storage_config=storage_config + ) + assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}"