diff --git a/ci/schema/bigquery.sql b/ci/schema/bigquery.sql index a9baa7fafbb5f..494a3116117a2 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 64cb04ca08214..4a7119f4b8299 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 0be6703b01480..f4b141d4888da 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 0ec19ea11bd28..272d8f9e94c1c 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 30ec15b568af6..fb72e07cc22bb 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 0419f4e357e93..101213ca2685d 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 1f3af9e95fd04..dc5b1b9eb8a11 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 0ca4403145e83..2092ed9d9b130 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 cfe8f27ac2e9c..541ec632872e7 100644 --- a/ibis/backends/bigquery/compiler.py +++ b/ibis/backends/bigquery/compiler.py @@ -381,6 +381,26 @@ 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.if_( + self.f.json_type(arg).eq("string"), self.f.lax_string(arg), NULL + ) + + def visit_UnwrapJSONInt64(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("number"), self.f.anon["safe.int64"](arg), NULL + ) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("number"), self.f.anon["safe.float64"](arg), NULL + ) + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.if_( + self.f.json_type(arg).eq("boolean"), self.f.anon["safe.bool"](arg), NULL + ) + 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 c7d4cd3c85ebb..2c2d0cc25cbe4 100644 --- a/ibis/backends/duckdb/compiler.py +++ b/ibis/backends/duckdb/compiler.py @@ -214,6 +214,36 @@ def visit_ToJSONMap(self, op, *, arg): def visit_ToJSONArray(self, op, *, arg): return self.visit_ToJSONMap(op, arg=arg) + 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_( + sg.or_(arg_type.eq("UBIGINT"), arg_type.eq("BIGINT")), + self.cast(arg, op.dtype), + NULL, + ) + + def visit_UnwrapJSONFloat64(self, op, *, arg): + arg_type = self.f.json_type(arg) + return self.if_( + sg.or_( + arg_type.eq("UBIGINT"), arg_type.eq("BIGINT"), arg_type.eq("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 return reduce(self.f.list_concat, arg) diff --git a/ibis/backends/mysql/compiler.py b/ibis/backends/mysql/compiler.py index ef636acde2294..a10a153eeb736 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 d3b2034518b48..fc6fb8ca264f4 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 ca346b6583f6b..f0ec815955fe9 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 0793ce9d5d834..31aa53ae01357 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 78061eaf6a1e8..75ea7a5c076f0 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 32e67df849514..b9f3dfc4acb23 100644 --- a/ibis/backends/sqlite/compiler.py +++ b/ibis/backends/sqlite/compiler.py @@ -234,6 +234,36 @@ 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_( + sg.or_( + self.f.json_type(arg).eq("integer"), self.f.json_type(arg).eq("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 + sg.or_(self.f.json_type(arg).eq("true"), self.f.json_type(arg).eq("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 3191ccc4ea01c..f8599945892b6 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 7130c124f82ad..3adc1aeaa2075 100644 --- a/ibis/backends/tests/test_client.py +++ b/ibis/backends/tests/test_client.py @@ -1579,6 +1579,10 @@ def test_json_to_pyarrow(con): None, [42, 47, 55], [], + "a", + "", + "b", + None, ] expected = {json.dumps(val) for val in expected} @@ -1586,5 +1590,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 5e16da75c25e9..3e4e156f047bb 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 +from ibis.backends.tests.errors import TrinoUserError + pytestmark = [ pytest.mark.never(["impala"], reason="doesn't support JSON and never will"), pytest.mark.notyet(["clickhouse"], reason="upstream is broken"), @@ -64,7 +66,10 @@ def test_json_getitem_array(json_t): @pytest.mark.notyet(["bigquery", "sqlite"], reason="doesn't support maps") @pytest.mark.notyet(["postgres"], reason="only supports map") @pytest.mark.notyet( - ["pyspark", "trino", "flink"], reason="should work but doesn't deserialize JSON" + ["pyspark", "flink"], reason="should work but doesn't deserialize JSON" +) +@pytest.mark.notyet( + ["trino"], raises=TrinoUserError, reason="connector can't deserialize JSON" ) def test_json_map(backend, json_t): expr = json_t.js.map.name("res") @@ -77,6 +82,10 @@ def test_json_map(backend, json_t): None, None, None, + None, + None, + None, + None, ], dtype="object", name="res", @@ -87,13 +96,52 @@ def test_json_map(backend, json_t): @pytest.mark.notimpl(["dask", "mysql", "pandas", "risingwave"]) @pytest.mark.notyet(["sqlite"], reason="doesn't support arrays") @pytest.mark.notyet( - ["pyspark", "trino", "flink"], reason="should work but doesn't deserialize JSON" + ["pyspark", "flink"], reason="should work but doesn't deserialize JSON" ) @pytest.mark.notyet(["bigquery"], reason="doesn't allow null in arrays") +@pytest.mark.notyet( + ["trino"], raises=TrinoUserError, reason="connector can't deserialize JSON" +) 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, None, None, None], + 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.notyet( + ["trino"], raises=TrinoUserError, reason="connector can't deserialize JSON" +) +@pytest.mark.parametrize( + ("prop", "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"], +) +def test_json_unwrap(backend, json_t, prop, expected_data): + expr = getattr(json_t.js, prop).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 bcbb0046457f9..686762148a365 100644 --- a/ibis/backends/trino/compiler.py +++ b/ibis/backends/trino/compiler.py @@ -173,6 +173,21 @@ 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(arg, 'strict $?($.type() == "string")') + + def visit_UnwrapJSONInt64(self, op, *, arg): + return self.cast( + self.f.json_value(arg, 'strict $?($.type() == "number")'), op.dtype + ) + + visit_UnwrapJSONFloat64 = visit_UnwrapJSONInt64 + + def visit_UnwrapJSONBoolean(self, op, *, arg): + return self.cast( + self.f.json_value(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 ad1307df65cff..5a259d804f4b4 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 d4d62f732cb03..4981b14eccf1e 100644 --- a/ibis/expr/types/json.py +++ b/ibis/expr/types/json.py @@ -124,6 +124,223 @@ 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"}'}, + ... ] + >>> 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'} │ + └─────────┴─────────────────────────────────────────┘ + >>> 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 │ + └─────────┴─────────────────────────────────────────┴─────────┘ + """ + 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):