diff --git a/ci/schema/bigquery.sql b/ci/schema/bigquery.sql index a9baa7fafbb5..494a3116117a 100644 --- a/ci/schema/bigquery.sql +++ b/ci/schema/bigquery.sql @@ -65,7 +65,15 @@ INSERT INTO {dataset}.json_t VALUES (JSON '{{"a":"foo", "c":null}}'), (JSON 'null'), (JSON '[42,47,55]'), - (JSON '[]'); + (JSON '[]'), + (JSON '"a"'), + (JSON '""'), + (JSON '"b"'), + (NULL), + (JSON 'true'), + (JSON 'false'), + (JSON '42'), + (JSON '37.37'); LOAD DATA OVERWRITE {dataset}.functional_alltypes ( diff --git a/ci/schema/duckdb.sql b/ci/schema/duckdb.sql index 64cb04ca0821..4a7119f4b829 100644 --- a/ci/schema/duckdb.sql +++ b/ci/schema/duckdb.sql @@ -29,7 +29,7 @@ INSERT INTO struct VALUES (NULL), ({'a': 3.0, 'b': 'orange', 'c': NULL}); -CREATE OR REPLACE TABLE json_t (js TEXT); +CREATE OR REPLACE TABLE json_t (js JSON); INSERT INTO json_t VALUES ('{"a": [1,2,3,4], "b": 1}'), @@ -37,7 +37,15 @@ INSERT INTO json_t VALUES ('{"a":"foo", "c":null}'), ('null'), ('[42,47,55]'), - ('[]'); + ('[]'), + ('"a"'), + ('""'), + ('"b"'), + (NULL), + ('true'), + ('false'), + ('42'), + ('37.37'); CREATE OR REPLACE TABLE win (g TEXT, x BIGINT NOT NULL, y BIGINT); INSERT INTO win VALUES diff --git a/ci/schema/mysql.sql b/ci/schema/mysql.sql index 0be6703b0148..f4b141d4888d 100644 --- a/ci/schema/mysql.sql +++ b/ci/schema/mysql.sql @@ -108,7 +108,15 @@ INSERT INTO json_t VALUES ('{"a":"foo", "c":null}'), ('null'), ('[42,47,55]'), - ('[]'); + ('[]'), + ('"a"'), + ('""'), + ('"b"'), + (NULL), + ('true'), + ('false'), + ('42'), + ('37.37'); DROP TABLE IF EXISTS win CASCADE; diff --git a/ci/schema/postgres.sql b/ci/schema/postgres.sql index 0ec19ea11bd2..272d8f9e94c1 100644 --- a/ci/schema/postgres.sql +++ b/ci/schema/postgres.sql @@ -273,7 +273,15 @@ INSERT INTO json_t VALUES ('{"a":"foo", "c":null}'), ('null'), ('[42,47,55]'), - ('[]'); + ('[]'), + ('"a"'), + ('""'), + ('"b"'), + (NULL), + ('true'), + ('false'), + ('42'), + ('37.37'); DROP TABLE IF EXISTS win CASCADE; CREATE TABLE win (g TEXT, x BIGINT NOT NULL, y BIGINT); diff --git a/ci/schema/risingwave.sql b/ci/schema/risingwave.sql index 30ec15b568af..fb72e07cc22b 100644 --- a/ci/schema/risingwave.sql +++ b/ci/schema/risingwave.sql @@ -165,7 +165,15 @@ INSERT INTO "json_t" VALUES ('{"a":"foo", "c":null}'), ('null'), ('[42,47,55]'), - ('[]'); + ('[]'), + ('"a"'), + ('""'), + ('"b"'), + (NULL), + ('true'), + ('false'), + ('42'), + ('37.37'); DROP TABLE IF EXISTS "win" CASCADE; CREATE TABLE "win" ("g" TEXT, "x" BIGINT, "y" BIGINT); diff --git a/ci/schema/snowflake.sql b/ci/schema/snowflake.sql index 0419f4e357e9..101213ca2685 100644 --- a/ci/schema/snowflake.sql +++ b/ci/schema/snowflake.sql @@ -131,7 +131,15 @@ INSERT INTO "json_t" ("js") SELECT parse_json('{"a":"foo", "c":null}') UNION SELECT parse_json('null') UNION SELECT parse_json('[42,47,55]') UNION - SELECT parse_json('[]'); + SELECT parse_json('[]') UNION + SELECT parse_json('"a"') UNION + SELECT parse_json('""') UNION + SELECT parse_json('"b"') UNION + SELECT NULL UNION + SELECT parse_json('true') UNION + SELECT parse_json('false') UNION + SELECT parse_json('42') UNION + SELECT parse_json('37.37'); CREATE OR REPLACE TABLE "win" ("g" TEXT, "x" BIGINT NOT NULL, "y" BIGINT); INSERT INTO "win" VALUES diff --git a/ci/schema/sqlite.sql b/ci/schema/sqlite.sql index 1f3af9e95fd0..dc5b1b9eb8a1 100644 --- a/ci/schema/sqlite.sql +++ b/ci/schema/sqlite.sql @@ -109,7 +109,15 @@ INSERT INTO json_t VALUES ('{"a":"foo", "c":null}'), ('null'), ('[42,47,55]'), - ('[]'); + ('[]'), + ('"a"'), + ('""'), + ('"b"'), + (NULL), + ('true'), + ('false'), + ('42'), + ('37.37'); DROP TABLE IF EXISTS win; CREATE TABLE win (g TEXT, x BIGINT NOT NULL, y BIGINT); diff --git a/ci/schema/trino.sql b/ci/schema/trino.sql index 0ca4403145e8..2092ed9d9b13 100644 --- a/ci/schema/trino.sql +++ b/ci/schema/trino.sql @@ -168,7 +168,15 @@ INSERT INTO memory.default.json_t VALUES (JSON '{"a":"foo", "c":null}'), (JSON 'null'), (JSON '[42,47,55]'), - (JSON '[]'); + (JSON '[]'), + (JSON '"a"'), + (JSON '""'), + (JSON '"b"'), + (NULL), + (JSON 'true'), + (JSON 'false'), + (JSON '42'), + (JSON '37.37'); DROP TABLE IF EXISTS win; CREATE TABLE win (g VARCHAR, x BIGINT, y BIGINT); diff --git a/ibis/backends/bigquery/compiler.py b/ibis/backends/bigquery/compiler.py index cfe8f27ac2e9..c400a646c7f9 100644 --- a/ibis/backends/bigquery/compiler.py +++ b/ibis/backends/bigquery/compiler.py @@ -381,6 +381,18 @@ def visit_Cast(self, op, *, arg, to): def visit_JSONGetItem(self, op, *, arg, index): return arg[index] + def visit_UnwrapJSONString(self, op, *, arg): + return self.f.anon["safe.string"](arg) + + def visit_UnwrapJSONInt64(self, op, *, arg): + return self.f.anon["safe.int64"](arg) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + return self.f.anon["safe.float64"](arg) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.f.anon["safe.bool"](arg) + def visit_ExtractEpochSeconds(self, op, *, arg): return self.f.unix_seconds(arg) diff --git a/ibis/backends/duckdb/compiler.py b/ibis/backends/duckdb/compiler.py index c7d4cd3c85eb..4383be8f2978 100644 --- a/ibis/backends/duckdb/compiler.py +++ b/ibis/backends/duckdb/compiler.py @@ -209,10 +209,42 @@ def visit_MapContains(self, op, *, arg, key): return self.f.len(self.f.element_at(arg, key)).neq(0) def visit_ToJSONMap(self, op, *, arg): - return sge.TryCast(this=arg, to=self.type_mapper.from_ibis(op.dtype)) + return self.if_( + self.f.json_type(arg).eq("OBJECT"), + self.cast(self.cast(arg, dt.json), op.dtype), + NULL, + ) def visit_ToJSONArray(self, op, *, arg): - return self.visit_ToJSONMap(op, arg=arg) + return self.if_( + self.f.json_type(arg).eq("ARRAY"), + self.cast(self.cast(arg, dt.json), op.dtype), + NULL, + ) + + def visit_UnwrapJSONString(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("VARCHAR"), + self.f.json_extract_string(arg, "$"), + NULL, + ) + + def visit_UnwrapJSONInt64(self, op, *, arg): + arg_type = self.f.json_type(arg) + return self.if_( + arg_type.isin("UBIGINT", "BIGINT"), self.cast(arg, op.dtype), NULL + ) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + arg_type = self.f.json_type(arg) + return self.if_( + arg_type.isin("UBIGINT", "BIGINT", "DOUBLE"), self.cast(arg, op.dtype), NULL + ) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("BOOLEAN"), self.cast(arg, op.dtype), NULL + ) def visit_ArrayConcat(self, op, *, arg): # TODO(cpcloud): map ArrayConcat to this in sqlglot instead of here diff --git a/ibis/backends/mysql/compiler.py b/ibis/backends/mysql/compiler.py index ef636acde229..624c37cb2049 100644 --- a/ibis/backends/mysql/compiler.py +++ b/ibis/backends/mysql/compiler.py @@ -342,3 +342,25 @@ def visit_TimestampAdd(self, op, *, left, right): this=right.this * 1_000, unit=sge.Var(this="MICROSECOND") ) return self.f.date_add(left, right, dialect=self.dialect) + + def visit_UnwrapJSONString(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("STRING"), self.f.json_unquote(arg), NULL + ) + + def visit_UnwrapJSONInt64(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("INTEGER"), self.cast(arg, op.dtype), NULL + ) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + return self.if_( + self.f.json_type(arg).isin("DOUBLE", "INTEGER"), + self.cast(arg, op.dtype), + NULL, + ) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("BOOLEAN"), self.if_(arg.eq("true"), 1, 0), NULL + ) diff --git a/ibis/backends/postgres/compiler.py b/ibis/backends/postgres/compiler.py index d3b2034518b4..fc6fb8ca264f 100644 --- a/ibis/backends/postgres/compiler.py +++ b/ibis/backends/postgres/compiler.py @@ -325,6 +325,53 @@ def visit_StructField(self, op, *, arg, field): op.dtype, ) + def visit_UnwrapJSONString(self, op, *, arg): + return self.if_( + self.f.json_typeof(arg).eq("string"), + self.f.json_extract_path_text( + arg, + # this is apparently how you pass in no additional arguments to + # a variadic function, see the "Variadic Function Resolution" + # section in + # https://www.postgresql.org/docs/current/typeconv-func.html + sge.Var(this="VARIADIC ARRAY[]::TEXT[]"), + ), + NULL, + ) + + def visit_UnwrapJSONInt64(self, op, *, arg): + text = self.f.json_extract_path_text( + arg, sge.Var(this="VARIADIC ARRAY[]::TEXT[]") + ) + return self.if_( + self.f.json_typeof(arg).eq("number"), + self.cast( + self.if_(self.f.regexp_like(text, r"^\d+$", "g"), text, NULL), + op.dtype, + ), + NULL, + ) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + text = self.f.json_extract_path_text( + arg, sge.Var(this="VARIADIC ARRAY[]::TEXT[]") + ) + return self.if_( + self.f.json_typeof(arg).eq("number"), self.cast(text, op.dtype), NULL + ) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.if_( + self.f.json_typeof(arg).eq("boolean"), + self.cast( + self.f.json_extract_path_text( + arg, sge.Var(this="VARIADIC ARRAY[]::TEXT[]") + ), + op.dtype, + ), + NULL, + ) + def visit_StructColumn(self, op, *, names, values): return self.f.row(*map(self.cast, values, op.dtype.types)) diff --git a/ibis/backends/pyspark/__init__.py b/ibis/backends/pyspark/__init__.py index ca346b6583f6..f0ec815955fe 100644 --- a/ibis/backends/pyspark/__init__.py +++ b/ibis/backends/pyspark/__init__.py @@ -11,6 +11,7 @@ from pyspark import SparkConf from pyspark.sql import DataFrame, SparkSession from pyspark.sql.functions import PandasUDFType, pandas_udf +from pyspark.sql.types import BooleanType, DoubleType, LongType, StringType import ibis.common.exceptions as com import ibis.config @@ -40,6 +41,47 @@ def normalize_filenames(source_list): return list(map(util.normalize_filename, source_list)) +@pandas_udf(returnType=DoubleType(), functionType=PandasUDFType.SCALAR) +def unwrap_json_float(s: pd.Series) -> pd.Series: + import json + + import pandas as pd + + def nullify_type_mismatched_value(raw): + if pd.isna(raw): + return None + + value = json.loads(raw) + # exact type check because we want to distinguish between integer + # and booleans and bool is a subclass of int + return value if type(value) in (float, int) else None + + return s.map(nullify_type_mismatched_value) + + +def unwrap_json(typ): + import json + + import pandas as pd + + type_mapping = {str: StringType(), int: LongType(), bool: BooleanType()} + + @pandas_udf(returnType=type_mapping[typ], functionType=PandasUDFType.SCALAR) + def unwrap(s: pd.Series) -> pd.Series: + def nullify_type_mismatched_value(raw): + if pd.isna(raw): + return None + + value = json.loads(raw) + # exact type check because we want to distinguish between integer + # and booleans and bool is a subclass of int + return value if type(value) == typ else None + + return s.map(nullify_type_mismatched_value) + + return unwrap + + class _PySparkCursor: """Spark cursor. @@ -252,6 +294,10 @@ def _register_udfs(self, expr: ir.Expr) -> None: spark_udf = pandas_udf(udf_func, udf_return, PandasUDFType.GROUPED_AGG) self._session.udf.register(udf_name, spark_udf) + for typ in (str, int, bool): + self._session.udf.register(f"unwrap_json_{typ.__name__}", unwrap_json(typ)) + self._session.udf.register("unwrap_json_float", unwrap_json_float) + def _register_in_memory_table(self, op: ops.InMemoryTable) -> None: schema = PySparkSchema.from_ibis(op.schema) df = self._session.createDataFrame(data=op.data.to_frame(), schema=schema) diff --git a/ibis/backends/pyspark/compiler.py b/ibis/backends/pyspark/compiler.py index 0793ce9d5d83..31aa53ae0135 100644 --- a/ibis/backends/pyspark/compiler.py +++ b/ibis/backends/pyspark/compiler.py @@ -77,6 +77,10 @@ class PySparkCompiler(SQLGlotCompiler): ops.MapMerge: "map_concat", ops.MapKeys: "map_keys", ops.MapValues: "map_values", + ops.UnwrapJSONString: "unwrap_json_str", + ops.UnwrapJSONInt64: "unwrap_json_int", + ops.UnwrapJSONFloat64: "unwrap_json_float", + ops.UnwrapJSONBoolean: "unwrap_json_bool", } def _aggregate(self, funcname: str, *args, where): diff --git a/ibis/backends/snowflake/compiler.py b/ibis/backends/snowflake/compiler.py index 78061eaf6a1e..75ea7a5c076f 100644 --- a/ibis/backends/snowflake/compiler.py +++ b/ibis/backends/snowflake/compiler.py @@ -179,6 +179,18 @@ def visit_ToJSONMap(self, op, *, arg): def visit_ToJSONArray(self, op, *, arg): return self.if_(self.f.is_array(arg), arg, NULL) + def visit_UnwrapJSONString(self, op, *, arg): + return self.if_(self.f.is_varchar(arg), self.f.as_varchar(arg), NULL) + + def visit_UnwrapJSONInt64(self, op, *, arg): + return self.if_(self.f.is_integer(arg), self.f.as_integer(arg), NULL) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + return self.if_(self.f.is_double(arg), self.f.as_double(arg), NULL) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.if_(self.f.is_boolean(arg), self.f.as_boolean(arg), NULL) + def visit_IsNan(self, op, *, arg): return arg.eq(self.NAN) diff --git a/ibis/backends/sqlite/compiler.py b/ibis/backends/sqlite/compiler.py index eed2430502d4..42effea6c4fb 100644 --- a/ibis/backends/sqlite/compiler.py +++ b/ibis/backends/sqlite/compiler.py @@ -234,6 +234,34 @@ def _visit_arg_reduction(self, func, op, *, arg, key, where): agg = self._aggregate(func, key, where=cond) return self.f.anon.json_extract(self.f.json_array(arg, agg), "$[0]") + def visit_UnwrapJSONString(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("text"), self.f.json_extract_scalar(arg, "$"), NULL + ) + + def visit_UnwrapJSONInt64(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("integer"), + self.cast(self.f.json_extract_scalar(arg, "$"), op.dtype), + NULL, + ) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + return self.if_( + self.f.json_type(arg).isin("integer", "real"), + self.cast(self.f.json_extract_scalar(arg, "$"), op.dtype), + NULL, + ) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.if_( + # isin doesn't work here, with a strange error from sqlite about a + # misused row value + self.f.json_type(arg).isin("true", "false"), + self.cast(self.f.json_extract_scalar(arg, "$"), dt.int64), + NULL, + ) + def visit_Variance(self, op, *, arg, how, where): return self._aggregate(f"_ibis_var_{op.how}", arg, where=where) diff --git a/ibis/backends/tests/data.py b/ibis/backends/tests/data.py index 3191ccc4ea01..f8599945892b 100644 --- a/ibis/backends/tests/data.py +++ b/ibis/backends/tests/data.py @@ -100,6 +100,14 @@ "null", "[42,47,55]", "[]", + '"a"', + '""', + '"b"', + None, + "true", + "false", + "42", + "37.37", ] } ) diff --git a/ibis/backends/tests/test_client.py b/ibis/backends/tests/test_client.py index 8920e2a2e176..3a5900d5e45e 100644 --- a/ibis/backends/tests/test_client.py +++ b/ibis/backends/tests/test_client.py @@ -1585,6 +1585,14 @@ def test_json_to_pyarrow(con): None, [42, 47, 55], [], + "a", + "", + "b", + None, + True, + False, + 42, + 37.37, ] expected = {json.dumps(val) for val in expected} @@ -1592,5 +1600,10 @@ def test_json_to_pyarrow(con): # loads and dumps so the string representation is the same json.dumps(json.loads(val)) for val in js.to_pylist() + # proper null values must be ignored because they cannot be + # deserialized as JSON + # + # they exist in the json_t table, so the `js` value contains them + if val is not None } assert result == expected diff --git a/ibis/backends/tests/test_json.py b/ibis/backends/tests/test_json.py index 60ce6d4ce46e..1677ac094eb8 100644 --- a/ibis/backends/tests/test_json.py +++ b/ibis/backends/tests/test_json.py @@ -9,6 +9,8 @@ import pytest from packaging.version import parse as vparse +import ibis.expr.types as ir + pytestmark = [ pytest.mark.never(["impala"], reason="doesn't support JSON and never will"), pytest.mark.notyet(["clickhouse"], reason="upstream is broken"), @@ -74,10 +76,8 @@ def test_json_map(backend, json_t): {"a": [1, 2, 3, 4], "b": 1}, {"a": None, "b": 2}, {"a": "foo", "c": None}, - None, - None, - None, - ], + ] + + [None] * 11, dtype="object", name="res", ) @@ -94,6 +94,42 @@ def test_json_array(backend, json_t): expr = json_t.js.array.name("res") result = expr.execute() expected = pd.Series( - [None, None, None, None, [42, 47, 55], []], name="res", dtype="object" + [None, None, None, None, [42, 47, 55], []] + [None] * 8, + name="res", + dtype="object", ) backend.assert_series_equal(result, expected) + + +@pytest.mark.notyet( + ["sqlite"], + condition=vparse(sqlite3.sqlite_version) < vparse("3.38.0"), + reason="JSON not supported in SQLite < 3.38.0", +) +@pytest.mark.notimpl(["dask", "pandas", "risingwave"]) +@pytest.mark.notyet(["flink"], reason="should work but doesn't deserialize JSON") +@pytest.mark.parametrize( + ("typ", "expected_data"), + [ + ("str", [None] * 6 + ["a", "", "b"] + [None] * 5), + ("int", [None] * 12 + [42, None]), + ("float", [None] * 12 + [42.0, 37.37]), + ("bool", [None] * 10 + [True, False, None, None]), + ], + ids=["str", "int", "float", "bool"], +) +@pytest.mark.parametrize( + "expr_fn", [getattr, ir.JSONValue.unwrap_as], ids=["getattr", "unwrap_as"] +) +def test_json_unwrap(backend, json_t, typ, expected_data, expr_fn): + expr = expr_fn(json_t.js, typ).name("res") + result = expr.execute() + expected = pd.Series(expected_data, name="res", dtype="object") + backend.assert_series_equal( + result.replace(np.nan, None).fillna(pd.NA).sort_values().reset_index(drop=True), + expected.replace(np.nan, None) + .fillna(pd.NA) + .sort_values() + .reset_index(drop=True), + check_dtype=False, + ) diff --git a/ibis/backends/trino/compiler.py b/ibis/backends/trino/compiler.py index 4635de48e86c..db68746a0021 100644 --- a/ibis/backends/trino/compiler.py +++ b/ibis/backends/trino/compiler.py @@ -173,6 +173,35 @@ def visit_JSONGetItem(self, op, *, arg, index): fmt = "%d" if op.index.dtype.is_integer() else '"%s"' return self.f.json_extract(arg, self.f.format(f"$[{fmt}]", index)) + def visit_UnwrapJSONString(self, op, *, arg): + return self.f.json_value( + self.f.json_format(arg), 'strict $?($.type() == "string")' + ) + + def visit_UnwrapJSONInt64(self, op, *, arg): + value = self.f.json_value( + self.f.json_format(arg), 'strict $?($.type() == "number")' + ) + return self.cast( + self.if_(self.f.regexp_like(value, r"^\d+$"), value, NULL), op.dtype + ) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + return self.cast( + self.f.json_value( + self.f.json_format(arg), 'strict $?($.type() == "number")' + ), + op.dtype, + ) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.cast( + self.f.json_value( + self.f.json_format(arg), 'strict $?($.type() == "boolean")' + ), + op.dtype, + ) + def visit_DayOfWeekIndex(self, op, *, arg): return self.cast( sge.paren(self.f.day_of_week(arg) + 6, copy=False) % 7, op.dtype diff --git a/ibis/expr/operations/json.py b/ibis/expr/operations/json.py index ad1307df65cf..5a259d804f4b 100644 --- a/ibis/expr/operations/json.py +++ b/ibis/expr/operations/json.py @@ -30,3 +30,35 @@ class ToJSONMap(Value): dtype = dt.Map(dt.string, dt.json) shape = rlz.shape_like("arg") + + +@public +class UnwrapJSONString(Value): + arg: Value[dt.JSON] + + dtype = dt.string + shape = rlz.shape_like("arg") + + +@public +class UnwrapJSONInt64(Value): + arg: Value[dt.JSON] + + dtype = dt.int64 + shape = rlz.shape_like("arg") + + +@public +class UnwrapJSONFloat64(Value): + arg: Value[dt.JSON] + + dtype = dt.float64 + shape = rlz.shape_like("arg") + + +@public +class UnwrapJSONBoolean(Value): + arg: Value[dt.JSON] + + dtype = dt.boolean + shape = rlz.shape_like("arg") diff --git a/ibis/expr/types/json.py b/ibis/expr/types/json.py index d4d62f732cb0..f741ebdc9848 100644 --- a/ibis/expr/types/json.py +++ b/ibis/expr/types/json.py @@ -6,6 +6,8 @@ from public import public +import ibis.common.exceptions as exc +import ibis.expr.datatypes as dt import ibis.expr.operations as ops from ibis.expr.types import Column, Scalar, Value @@ -98,6 +100,155 @@ def __getitem__( """ return ops.JSONGetItem(self, key).to_expr() + def unwrap_as(self, dtype: dt.DataType | str) -> ir.Value: + """Unwrap JSON into a specific data type. + + Returns + ------- + Value + An Ibis expression of a more specific type than JSON + + Examples + -------- + >>> import ibis + >>> ibis.options.interactive = True + >>> data = { + ... "jstring": ['"a"', '""', None, "null"], + ... "jbool": ["true", "false", "null", None], + ... "jint": ["1", "null", None, "2"], + ... "jfloat": ["42.42", None, "null", "37.37"], + ... "jmap": ['{"a": 1}', "null", None, "{}"], + ... "jarray": ["[]", "null", None, '[{},"1",2]'], + ... } + >>> t = ibis.memtable(data, schema=dict.fromkeys(data.keys(), "json")) + >>> t + ┏━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┳━━━┓ + ┃ jstring ┃ jbool ┃ jint ┃ … ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━╇━━━┩ + │ json │ json │ json │ … │ + ├──────────────────────┼──────────────────────┼──────────────────────┼───┤ + │ 'a' │ True │ 1 │ … │ + │ '' │ False │ None │ … │ + │ NULL │ None │ NULL │ … │ + │ None │ NULL │ 2 │ … │ + └──────────────────────┴──────────────────────┴──────────────────────┴───┘ + >>> t.select(unwrapped=t.jstring.unwrap_as(str), original=t.jstring) + ┏━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ unwrapped ┃ original ┃ + ┡━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━┩ + │ string │ json │ + ├───────────┼──────────────────────┤ + │ a │ 'a' │ + │ ~ │ '' │ + │ NULL │ NULL │ + │ NULL │ None │ + └───────────┴──────────────────────┘ + >>> t.select(unwrapped=t.jbool.unwrap_as("bool"), original=t.jbool) + ┏━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ unwrapped ┃ original ┃ + ┡━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━┩ + │ boolean │ json │ + ├───────────┼──────────────────────┤ + │ True │ True │ + │ False │ False │ + │ NULL │ None │ + │ NULL │ NULL │ + └───────────┴──────────────────────┘ + >>> t.select( + ... unwrapped_int64=t.jint.unwrap_as("int64"), + ... unwrapped_int32=t.jint.unwrap_as("int32"), + ... original=t.jint, + ... ) + ┏━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ unwrapped_int64 ┃ unwrapped_int32 ┃ original ┃ + ┡━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━┩ + │ int64 │ int32 │ json │ + ├─────────────────┼─────────────────┼──────────────────────┤ + │ 1 │ 1 │ 1 │ + │ NULL │ NULL │ None │ + │ NULL │ NULL │ NULL │ + │ 2 │ 2 │ 2 │ + └─────────────────┴─────────────────┴──────────────────────┘ + + You can cast to a more specific type than the types available in standards-compliant JSON. + + Here's an example of casting JSON numbers to `float32`: + + >>> t.select(unwrapped=t.jfloat.unwrap_as("float32"), original=t.jfloat) + ┏━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ unwrapped ┃ original ┃ + ┡━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━┩ + │ float32 │ json │ + ├───────────┼──────────────────────┤ + │ 42.419998 │ 42.42 │ + │ NULL │ NULL │ + │ NULL │ None │ + │ 37.369999 │ 37.37 │ + └───────────┴──────────────────────┘ + + You can cast JSON objects to a more specific `map` type: + + >>> t.select(unwrapped=t.jmap.unwrap_as("map"), original=t.jmap) + ┏━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ unwrapped ┃ original ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━┩ + │ map │ json │ + ├──────────────────────┼──────────────────────┤ + │ {'a': 1} │ {'a': 1} │ + │ NULL │ None │ + │ NULL │ NULL │ + │ {} │ {} │ + └──────────────────────┴──────────────────────┘ + + You can cast JSON arrays to an array type as well. In this case the + array values don't have a single element type so we cast to + `array`. + + >>> t.select(unwrapped=t.jarray.unwrap_as("array"), original=t.jarray) + ┏━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ unwrapped ┃ original ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━┩ + │ array │ json │ + ├───────────────────────┼──────────────────────┤ + │ [] │ [] │ + │ NULL │ None │ + │ NULL │ NULL │ + │ ['{}', '"1"', ... +1] │ [{...}, '1', ... +1] │ + └───────────────────────┴──────────────────────┘ + + See Also + -------- + [`JSONValue.str`](#ibis.expr.types.json.JSONValue.str) + [`JSONValue.int`](#ibis.expr.types.json.JSONValue.int) + [`JSONValue.float`](#ibis.expr.types.json.JSONValue.float) + [`JSONValue.bool`](#ibis.expr.types.json.JSONValue.bool) + [`JSONValue.map`](#ibis.expr.types.json.JSONValue.map) + [`JSONValue.array`](#ibis.expr.types.json.JSONValue.array) + [`Value.cast`](#ibis.expr.types.generic.Value.cast) + """ + dtype = dt.dtype(dtype) + if dtype.is_string(): + return self.str + elif dtype.is_boolean(): + return self.bool + elif dtype.is_integer(): + i = self.int + return i.cast(dtype) if i.type() != dtype else i + elif dtype.is_floating(): + f = self.float + return f.cast(dtype) if f.type() != dtype else f + elif dtype.is_map(): + m = self.map + return m.cast(dtype) if m.type() != dtype else m + elif dtype.is_array(): + a = self.array + return a.cast(dtype) if a.type() != dtype else a + else: + raise exc.IbisTypeError( + f"Data type {dtype} is unsupported for unwrapping JSON values. Supported " + "data types are strings, integers, floats, booleans, maps, and arrays." + ) + @property def map(self) -> ir.MapValue: """Cast JSON to a map of string to JSON. @@ -124,6 +275,226 @@ def array(self) -> ir.ArrayValue: """ return ops.ToJSONArray(self).to_expr() + @property + def int(self) -> ir.IntegerValue: + """Unwrap a JSON value into a backend-native int. + + Any non-float JSON values are returned as `NULL`. + + Examples + -------- + >>> import json, ibis + >>> ibis.options.interactive = True + >>> data = [ + ... {"name": "Alice", "json_data": '{"last_name":"Smith","age":40}'}, + ... {"name": "Bob", "json_data": '{"last_name":"Jones", "age":39}'}, + ... {"name": "Charlie", "json_data": '{"last_name":"Davies","age":54}'}, + ... ] + >>> t = ibis.memtable(data, schema={"name": "string", "json_data": "json"}) + >>> t + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ name ┃ json_data ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ + │ string │ json │ + ├─────────┼────────────────────────────────────┤ + │ Alice │ {'last_name': 'Smith', 'age': 40} │ + │ Bob │ {'last_name': 'Jones', 'age': 39} │ + │ Charlie │ {'last_name': 'Davies', 'age': 54} │ + └─────────┴────────────────────────────────────┘ + >>> t.mutate(age=t.json_data["age"].int) + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━┓ + ┃ name ┃ json_data ┃ age ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━┩ + │ string │ json │ int64 │ + ├─────────┼────────────────────────────────────┼───────┤ + │ Alice │ {'last_name': 'Smith', 'age': 40} │ 40 │ + │ Bob │ {'last_name': 'Jones', 'age': 39} │ 39 │ + │ Charlie │ {'last_name': 'Davies', 'age': 54} │ 54 │ + └─────────┴────────────────────────────────────┴───────┘ + """ + return ops.UnwrapJSONInt64(self).to_expr() + + @property + def float(self) -> ir.FloatingValue: + """Unwrap a JSON value into a backend-native float. + + Any non-float JSON values are returned as `NULL`. + + ::: {.callout-warning} + ## The `float` property is lax with respect to integers + + The `float` property will attempt to coerce integers to floating point numbers. + ::: + + Examples + -------- + >>> import json, ibis + >>> ibis.options.interactive = True + >>> data = [ + ... {"name": "Alice", "json_data": '{"last_name":"Smith","salary":42.42}'}, + ... {"name": "Bob", "json_data": '{"last_name":"Jones", "salary":37.37}'}, + ... {"name": "Charlie", "json_data": '{"last_name":"Davies","salary":"NA"}'}, + ... {"name": "Joan", "json_data": '{"last_name":"Davies","salary":78}'}, + ... ] + >>> t = ibis.memtable(data, schema={"name": "string", "json_data": "json"}) + >>> t + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ name ┃ json_data ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ + │ string │ json │ + ├─────────┼─────────────────────────────────────────┤ + │ Alice │ {'last_name': 'Smith', 'salary': 42.42} │ + │ Bob │ {'last_name': 'Jones', 'salary': 37.37} │ + │ Charlie │ {'last_name': 'Davies', 'salary': 'NA'} │ + │ Joan │ {'last_name': 'Davies', 'salary': 78} │ + └─────────┴─────────────────────────────────────────┘ + >>> t.mutate(salary=t.json_data["salary"].float) + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━┓ + ┃ name ┃ json_data ┃ salary ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━┩ + │ string │ json │ float64 │ + ├─────────┼─────────────────────────────────────────┼─────────┤ + │ Alice │ {'last_name': 'Smith', 'salary': 42.42} │ 42.42 │ + │ Bob │ {'last_name': 'Jones', 'salary': 37.37} │ 37.37 │ + │ Charlie │ {'last_name': 'Davies', 'salary': 'NA'} │ NULL │ + │ Joan │ {'last_name': 'Davies', 'salary': 78} │ 78.00 │ + └─────────┴─────────────────────────────────────────┴─────────┘ + """ + return ops.UnwrapJSONFloat64(self).to_expr() + + @property + def bool(self) -> ir.BooleanValue: + """Unwrap a JSON value into a backend-native boolean. + + Any non-boolean JSON values are returned as `NULL`. + + Examples + -------- + >>> import json, ibis + >>> ibis.options.interactive = True + >>> data = [ + ... {"name": "Alice", "json_data": '{"last_name":"Smith","is_bot":false}'}, + ... {"name": "Bob", "json_data": '{"last_name":"Jones","is_bot":true}'}, + ... {"name": "Charlie", "json_data": '{"last_name":"Davies","is_bot":false}'}, + ... ] + >>> t = ibis.memtable(data, schema={"name": "string", "json_data": "json"}) + >>> t + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ name ┃ json_data ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ + │ string │ json │ + ├─────────┼──────────────────────────────────────────┤ + │ Alice │ {'last_name': 'Smith', 'is_bot': False} │ + │ Bob │ {'last_name': 'Jones', 'is_bot': True} │ + │ Charlie │ {'last_name': 'Davies', 'is_bot': False} │ + └─────────┴──────────────────────────────────────────┘ + >>> t.mutate(is_bot=t.json_data["is_bot"].bool) + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━┓ + ┃ name ┃ json_data ┃ is_bot ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━┩ + │ string │ json │ boolean │ + ├─────────┼──────────────────────────────────────────┼─────────┤ + │ Alice │ {'last_name': 'Smith', 'is_bot': False} │ False │ + │ Bob │ {'last_name': 'Jones', 'is_bot': True} │ True │ + │ Charlie │ {'last_name': 'Davies', 'is_bot': False} │ False │ + └─────────┴──────────────────────────────────────────┴─────────┘ + """ + return ops.UnwrapJSONBoolean(self).to_expr() + + @property + def str(self) -> ir.StringValue: + """Unwrap a JSON string into a backend-native string. + + Any non-string JSON values are returned as `NULL`. + + Returns + ------- + StringValue + A string expression + + Examples + -------- + >>> import json, ibis + >>> ibis.options.interactive = True + >>> t = ibis.memtable( + ... {"js": ['"a"', '"b"', "1", "{}", '[{"a": 1}]']}, + ... schema=ibis.schema(dict(js="json")), + ... ) + >>> t + ┏━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ js ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━┩ + │ json │ + ├──────────────────────┤ + │ 'a' │ + │ 'b' │ + │ 1 │ + │ {} │ + │ [{...}] │ + └──────────────────────┘ + >>> t.js.str + ┏━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ UnwrapJSONString(js) ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━┩ + │ string │ + ├──────────────────────┤ + │ a │ + │ b │ + │ NULL │ + │ NULL │ + │ NULL │ + └──────────────────────┘ + + Note the difference between `.string` and `.cast("string")`. + + The latter preserves quotes for JSON string values and returns a valid + JSON string. + + >>> t.js.cast("string") + ┏━━━━━━━━━━━━━━━━━━┓ + ┃ Cast(js, string) ┃ + ┡━━━━━━━━━━━━━━━━━━┩ + │ string │ + ├──────────────────┤ + │ "a" │ + │ "b" │ + │ 1 │ + │ {} │ + │ [{"a": 1}] │ + └──────────────────┘ + + Here's a more complex example with a table containing a JSON column + with nested fields. + + >>> data = [ + ... {"name": "Alice", "json_data": '{"last_name":"Smith"}'}, + ... {"name": "Bob", "json_data": '{"last_name":"Jones"}'}, + ... {"name": "Charlie", "json_data": '{"last_name":"Davies"}'}, + ... ] + >>> t = ibis.memtable(data, schema={"name": "string", "json_data": "json"}) + >>> t + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ name ┃ json_data ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━┩ + │ string │ json │ + ├─────────┼─────────────────────────┤ + │ Alice │ {'last_name': 'Smith'} │ + │ Bob │ {'last_name': 'Jones'} │ + │ Charlie │ {'last_name': 'Davies'} │ + └─────────┴─────────────────────────┘ + >>> t.mutate(last_name=t.json_data["last_name"].str) + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┓ + ┃ name ┃ json_data ┃ last_name ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━┩ + │ string │ json │ string │ + ├─────────┼─────────────────────────┼───────────┤ + │ Alice │ {'last_name': 'Smith'} │ Smith │ + │ Bob │ {'last_name': 'Jones'} │ Jones │ + │ Charlie │ {'last_name': 'Davies'} │ Davies │ + └─────────┴─────────────────────────┴───────────┘ + """ + return ops.UnwrapJSONString(self).to_expr() + @public class JSONScalar(Scalar, JSONValue):