Skip to content

GH-3414: Add parseJson to VariantBuilder for JSON-to-Variant conversion#3415

Merged
wgtmac merged 1 commit into
apache:masterfrom
gaurav7261:gh-3414-variant-parse-json
May 12, 2026
Merged

GH-3414: Add parseJson to VariantBuilder for JSON-to-Variant conversion#3415
wgtmac merged 1 commit into
apache:masterfrom
gaurav7261:gh-3414-variant-parse-json

Conversation

@gaurav7261

Copy link
Copy Markdown
Contributor

Rationale for this change

Every consumer of parquet-variant currently has to independently implement JSON-to-Variant parsing. Apache Spark has one in its common/variant module
(source), our Kafka Connect S3 sink connector had to write one, and any other project (Flink, Trino, DuckDB-Java, etc.) would need to do the
same. Since VariantBuilder already provides all the low-level append*() primitives, parseJson() is the natural completion of that API — a canonical, reusable entry point for the most common use
case: converting a JSON string into a Variant.

What changes are included in this PR?

  • parquet-variant/pom.xml: Added jackson-core (compile) and parquet-jackson (runtime) dependencies, following the same pattern as parquet-hadoop.
  • VariantBuilder.java: Added two public static methods:
    • parseJson(String json) — convenience method that creates a Jackson streaming parser internally.
    • parseJson(JsonParser parser) — for callers who already have a positioned parser (e.g., reading from a stream).
    • Internal helpers ported from Apache Spark's production VariantBuilder.buildJson:
      • buildJson() — recursive single-pass streaming parser handling OBJECT, ARRAY, STRING, NUMBER_INT, NUMBER_FLOAT, TRUE, FALSE, NULL.
      • appendSmallestLong() — selects the smallest integer type (BYTE/SHORT/INT/LONG) based on value range.
      • tryAppendDecimal() — decimal-first encoding for floating-point numbers; falls back to double only for scientific notation or values exceeding DECIMAL16 precision (38 digits).
  • TestVariantParseJson.java: 32 new tests covering all primitive types, objects (empty, simple, nested, null values, sorted keys, duplicate keys), arrays (empty, simple, nested, mixed types), and
    edge cases (unicode, escaped strings, deeply nested documents, scientific notation, integer overflow to decimal, malformed JSON).

Are these changes tested?

Yes. 32 new tests in TestVariantParseJson.

Are there any user-facing changes?

Yes. Two new public static methods on VariantBuilder:

  • VariantBuilder.parseJson(String json) — returns a Variant
  • VariantBuilder.parseJson(JsonParser parser) — returns a Variant
    These are additive API additions with no breaking changes to existing APIs.
    Closes parseJson implementation in VariantBuilder #3414

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@alamb @aihuaxu read https://parquet.apache.org/blog/2026/02/27/variant-type-in-apache-parquet-for-semi-structured-data/ and check the feasibility of having our S3 Sink connector write variant, found out that parseJson can be a better fit here, wdyt? is it making sense

gaurav7261 added a commit to gaurav7261/kafka-connect-storage-cloud that referenced this pull request Mar 8, 2026
…uctured fields

Write Kafka Connect JSON fields (Debezium CDC, Confluent Protobuf Struct,
custom messages, maps, arrays) as native Parquet VARIANT columns instead of
plain STRING, improving storage efficiency and query performance.

- Upgrade parquet-java to 1.17.0 for VARIANT logical type support - https://parquet.apache.org/blog/2026/02/27/variant-type-in-apache-parquet-for-semi-structured-data/
- Add config: parquet.variant.enabled, parquet.variant.connect.names,
  parquet.variant.field.names
- Auto-detect recursive schemas (google.protobuf.Struct) as VARIANT
- Stream JSON-to-Variant conversion ported from Apache Spark (PR also raised in parquet-java, once approved, code will be neat here in this repo: apache/parquet-java#3415)
- Unwrap Protobuf Struct/Value/ListValue/map-as-array to clean JSON
- Graceful fallback for non-JSON values (e.g. __debezium_unavailable_value)
- Feature is fully opt-in (disabled by default), zero impact on existing connectors
@gaurav7261

Copy link
Copy Markdown
Contributor Author

@Fokko can you please review, is it looking good to you?

@alamb

alamb commented Mar 11, 2026

Copy link
Copy Markdown
Contributor

@gaurav7261 gaurav7261 force-pushed the gh-3414-variant-parse-json branch from 1274733 to 643ac64 Compare March 11, 2026 17:49
@gaurav7261

Copy link
Copy Markdown
Contributor Author

@julienledem thanks for the call, I have added notice, please review

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@gszadovszky can you please review as well

@wgtmac

wgtmac commented Mar 17, 2026

Copy link
Copy Markdown
Member

I'm not familiar with this code yet but I think it is worth adding.

@emkornfield @gene-db @rdblue WDYT?

@steveloughran steveloughran left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the json parsing should be in a class alongside VariantBuilder, rather than inside it, some VariantJsonParser class.

  1. ensures that use of jackson classes are isolated; the core variant does not need jackson on the CP
  2. if there is anything which can be done to improve single line JSON content performance, then it can be done there as it's life will span the whole file, rather than a row

Once it's split out, a new question surfaces: should it belong in parquet-jackson? Personally, I think it should

Comment thread NOTICE Outdated
This project includes code from Apache Spark with the following copyright
notice:

Apache Spark

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do cross-ASF projects need this credit? What I do think is good is ensure the original authors get credit in the final commit message

import org.junit.Assert;
import org.junit.Test;

public class TestVariantParseJson {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. nested object parsing?
  2. what about invalid json?
  • empty file
  • not a json file
  • incomplete
  • large json with many values

* @throws IOException if the JSON is malformed or an I/O error occurs
*/
public static Variant parseJson(String json) throws IOException {
try (JsonParser parser = JSON_FACTORY.createParser(json)) {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can be quite slow

* @return the parsed Variant
* @throws IOException if the JSON is malformed or an I/O error occurs
*/
public static Variant parseJson(String json) throws IOException {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was to suggest making CharSequence for feeding in from other places (string fields within avro, ...) but it looks like jackson 2 doesn't support that itself.

*/
public class VariantBuilder {

private static final JsonFactory JSON_FACTORY = new JsonFactory();

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

factory should be built with some constraints so that giving it a malicious json file should be rejected rather than trigger OOM problems, etc. From the javadocs.

JsonFactory f = JsonFactory.builder()
    .streamReadConstraints(
        StreamReadConstraints.builder()
            .maxNestingDepth(500)
            .maxStringLength(10_000_000)
            .maxDocumentLength(5_000_000)
            .build()
    )
    .build();

@gaurav7261 gaurav7261 force-pushed the gh-3414-variant-parse-json branch from 643ac64 to 08c3672 Compare March 19, 2026 21:44
@gaurav7261

Copy link
Copy Markdown
Contributor Author

@steveloughran can you please review again

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@steveloughran can you please review

@steveloughran steveloughran left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Commented. As I said before, I think this should go into parquet-jackson.

I see you've added jackson-core to the classpath. Is that because the parquet-jackson ones are shaded? If the class was in that module then it'd not be an issue.

of course, adding parquet-variant to the parquet-jackson module dependency list is its own complications, but I don't see any loops in the build.

Why not raise this PR on the dev list and bring up the "where should it go?" question.

Comment thread parquet-variant/pom.xml
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>${jackson.groupId}</groupId>

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

comes with the parquet-jackson module; no need to reimport at a different scope

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @steveloughran , email is sent to dev@parquet.apache.org, can you please check there

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @steveloughran , saw your reply on email, so all looks good to you in above PR

@steveloughran steveloughran left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I"m happy. Reviewed code and suggested some javadocs. Co-recursion is rare in production procedural code, so worth highlighting.

Test coverage good

}
}

private static void buildJsonObject(VariantBuilder builder, JsonParser parser) throws IOException {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: add javadoc explaining how this will co-recurse into buildJSON

builder.endObject();
}

private static void buildJsonArray(VariantBuilder builder, JsonParser parser) throws IOException {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same comment as above: mention co-recursion

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done @steveloughran , please check

@gaurav7261 gaurav7261 force-pushed the gh-3414-variant-parse-json branch from 08c3672 to 9c3dbab Compare April 14, 2026 12:30
@steveloughran

Copy link
Copy Markdown
Contributor

gaurav, you'll need to ask on the mail list for approval to run the workflows. Best to only do incremental changes on a PR

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@steveloughran i don't have access to email archive, @alamb created thread for me, @alamb can you please help here?

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@steveloughran can you please help here

@gaurav7261 gaurav7261 force-pushed the gh-3414-variant-parse-json branch from 9c3dbab to 8459204 Compare April 24, 2026 04:34
@gaurav7261

Copy link
Copy Markdown
Contributor Author

@wgtmac i have fixed mvn spotless, please rerun

…onversion

[apacheGH-3414] Extract JSON parsing into VariantJsonParser with StreamReadConstraints

  Move parseJson and helpers from VariantBuilder into dedicated VariantJsonParser
  class to isolate Jackson dependency. Add StreamReadConstraints to JsonFactory
  for safety against malicious input. Revert NOTICE (cross-ASF credit not needed).
  Add edge-case tests: empty input, non-JSON, incomplete array, large object.

  Ported from Apache Spark's VariantBuilder.parseJson.
@gaurav7261 gaurav7261 force-pushed the gh-3414-variant-parse-json branch from 8459204 to 814e6ef Compare April 24, 2026 14:15
@gaurav7261

gaurav7261 commented Apr 24, 2026

Copy link
Copy Markdown
Contributor Author

@wgtmac java version issue in local, now fixed with java 17 spotless,please rerun again, thanks

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@wgtmac can you please help run workflow

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@emkornfield CI passed, can we merge now please

@emkornfield

Copy link
Copy Markdown
Contributor

@gaurav7261 I think we need a committer to take a look. CC @wgtmac @Fokko @julienledem if you have time, otherwise I can take a look but I'm not super familiar with the java side of things.

@wgtmac wgtmac left a comment

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 from my side

Comment thread .gitignore
Comment thread parquet-variant/pom.xml
</dependency>
<dependency>
<groupId>${jackson.groupId}</groupId>
<artifactId>jackson-core</artifactId>

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this dependency change look good to you? @gszadovszky @Fokko

@gaurav7261

Copy link
Copy Markdown
Contributor Author

@wgtmac @Fokko can you please review and merge

@wgtmac wgtmac merged commit b9c11af into apache:master May 12, 2026
5 checks passed
@wgtmac

wgtmac commented May 12, 2026

Copy link
Copy Markdown
Member

Thanks @gaurav7261 for working on this and @steveloughran for the review!

@gaurav7261

gaurav7261 commented May 12, 2026

Copy link
Copy Markdown
Contributor Author

thanks @wgtmac, @steveloughran , i see release 1.17.1 is released 5 hrs ago, just want to know by when next release will get release so that i can use same in flink parquet and confluent kafka s3 sink connector

@wgtmac

wgtmac commented May 12, 2026

Copy link
Copy Markdown
Member

1.17.1 is a patched release with bug fixes only. For this feature, it is targeted in the 1.18.0 which I don't have an ETA yet. Perhaps you may want to reply to the relevant discussion thread in the dev@parquet.apache.org.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

parseJson implementation in VariantBuilder

5 participants