zhuqi-lucas commented on code in PR #19924: URL: https://github.com/apache/datafusion/pull/19924#discussion_r2727587680
########## datafusion/datasource-json/src/utils.rs: ########## @@ -0,0 +1,389 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Utility types for JSON processing + +use std::io::{BufRead, Read}; + +// ============================================================================ +// JsonArrayToNdjsonReader - Streaming JSON Array to NDJSON Converter +// ============================================================================ +// +// Architecture: +// +// ```text +// ┌─────────────────────────────────────────────────────────────┐ +// │ JSON Array File (potentially very large, e.g. 1GB) │ +// │ [{"a":1}, {"a":2}, {"a":3}, ...... {"a":1000000}] │ +// └─────────────────────────────────────────────────────────────┘ +// │ +// ▼ read small chunks at a time (e.g. 64KB) +// ┌───────────────────┐ +// │ JsonArrayToNdjson │ ← character substitution only: +// │ Reader │ '[' skip, ',' → '\n', ']' stop +// └───────────────────┘ +// │ +// ▼ outputs NDJSON format +// ┌───────────────────┐ +// │ Arrow Reader │ ← internal buffer, batch parsing +// │ batch_size=1024 │ +// └───────────────────┘ +// │ +// ▼ outputs RecordBatch for every batch_size rows +// ┌───────────────────┐ +// │ RecordBatch │ +// │ (1024 rows) │ +// └───────────────────┘ +// ``` +// +// Memory Efficiency: +// +// | Approach | Memory for 1GB file | Parse count | +// |---------------------------------------|---------------------|-------------| +// | Load entire file + serde_json | ~5GB | 3x | +// | Streaming with JsonArrayToNdjsonReader| ~few MB | 1x | +// + +/// Default buffer size for JsonArrayToNdjsonReader (64KB) +const DEFAULT_BUF_SIZE: usize = 64 * 1024; + +/// Parser state for JSON array streaming +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +enum JsonArrayState { + /// Initial state, looking for opening '[' + Start, + /// Inside the JSON array, processing objects + InArray, + /// Reached the closing ']', finished + Done, +} + +/// A streaming reader that converts JSON array format to NDJSON format. +/// +/// This reader wraps an underlying reader containing JSON array data +/// `[{...}, {...}, ...]` and transforms it on-the-fly to newline-delimited +/// JSON format that Arrow's JSON reader can process. +/// +/// Implements both `Read` and `BufRead` traits for compatibility with Arrow's +/// `ReaderBuilder::build()` which requires `BufRead`. +/// +/// # Transformation Rules +/// +/// - Skip leading `[` and whitespace before it +/// - Convert top-level `,` (between objects) to `\n` +/// - Skip whitespace at top level (between objects) +/// - Stop at trailing `]` +/// - Preserve everything inside objects (including nested `[`, `]`, `,`) +/// - Properly handle strings (ignore special chars inside quotes) +/// +/// # Example +/// +/// ```text +/// Input: [{"a":1}, {"b":[1,2]}, {"c":"x,y"}] +/// Output: {"a":1} +/// {"b":[1,2]} +/// {"c":"x,y"} +/// ``` +pub struct JsonArrayToNdjsonReader<R: Read> { + inner: R, + state: JsonArrayState, + /// Tracks nesting depth of `{` and `[` to identify top-level commas + depth: i32, + /// Whether we're currently inside a JSON string + in_string: bool, + /// Whether the next character is escaped (after `\`) + escape_next: bool, + /// Internal buffer for BufRead implementation + buffer: Vec<u8>, + /// Current position in the buffer + pos: usize, + /// Number of valid bytes in the buffer + filled: usize, +} + +impl<R: Read> JsonArrayToNdjsonReader<R> { + /// Create a new streaming reader that converts JSON array to NDJSON. + pub fn new(reader: R) -> Self { + Self { + inner: reader, + state: JsonArrayState::Start, + depth: 0, + in_string: false, + escape_next: false, + buffer: vec![0; DEFAULT_BUF_SIZE], + pos: 0, + filled: 0, + } + } + + /// Check if the JSON array was properly terminated. + /// + /// Returns an error if: + /// - Unbalanced braces/brackets (depth != 0) + /// - Unterminated string + /// - Missing closing `]` + pub fn validate_complete(&self) -> std::io::Result<()> { Review Comment: Yes @martin-g , added more unit tests in latest PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
