From 8ec607986fd38f357746fbaeabef2ce7ab3e501f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tim=20Swe=C3=B1a=20=28Swast=29?= Date: Tue, 6 May 2025 12:29:00 -0500 Subject: [PATCH 01/36] fix: rename columns with protected names such as `_TABLE_SUFFIX` in `to_gbq()` (#1691) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * perf: defer query in `read_gbq` with wildcard tables * remove obsolete comments * use sql node instead of ibis table node to keep select * from omitting pseudocolumns Fixes this code sample: import bigframes.pandas as bpd df = bpd.read_gbq("bigquery-public-data.google_analytics_sample.ga_sessions_*") df[df["_TABLE_SUFFIX"] == "20161204"].peek() * test with cache and to_gbq * rename columns before caching * remove unnecessary comment * add missing import * do not materialize _TABLE_SUFFIX * fix unit tests * correct number of columns in cache with offsets * fix formatting * 🦉 Updates from OwlBot post-processor See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md * revert datetime change, max_results change * add pseudocolumns to node * fix unit tests * actually fix unit tests * try to rename as part of compile * use correct node for table schema * revert pseudocolumn addition * add tests for fix for invalid columns * revert cached changes --------- Co-authored-by: Owl Bot --- bigframes/core/utils.py | 20 ++++++++ tests/system/small/test_dataframe_io.py | 68 +++++++++++++++++++++++++ 2 files changed, 88 insertions(+) diff --git a/bigframes/core/utils.py b/bigframes/core/utils.py index 495523d2fc..ba3fdcfd4b 100644 --- a/bigframes/core/utils.py +++ b/bigframes/core/utils.py @@ -147,6 +147,26 @@ def label_to_identifier(label: typing.Hashable, strict: bool = False) -> str: elif identifier[0].isdigit(): # first character must be letter or underscore identifier = "_" + identifier + + # Except in special circumstances (true anonymous query results tables), + # field names are not allowed to start with these (case-insensitive) + # prefixes. + # _PARTITION, _TABLE_, _FILE_, _ROW_TIMESTAMP, __ROOT__ and _COLIDENTIFIER + if any( + identifier.casefold().startswith(invalid_prefix.casefold()) + for invalid_prefix in ( + "_PARTITION", + "_TABLE_", + "_FILE_", + "_ROW_TIMESTAMP", + "__ROOT__", + "_COLIDENTIFIER", + ) + ): + # Remove leading _ character(s) to avoid collisions with preserved + # prefixes. + identifier = re.sub("^_+", "", identifier) + return identifier diff --git a/tests/system/small/test_dataframe_io.py b/tests/system/small/test_dataframe_io.py index e12db3f598..e210fed522 100644 --- a/tests/system/small/test_dataframe_io.py +++ b/tests/system/small/test_dataframe_io.py @@ -552,6 +552,74 @@ def test_to_gbq_w_duplicate_column_names( ) +def test_to_gbq_w_protected_column_names( + scalars_df_index, scalars_pandas_df_index, dataset_id +): + """ + Column names can't use any of the following prefixes: + + * _TABLE_ + * _FILE_ + * _PARTITION + * _ROW_TIMESTAMP + * __ROOT__ + * _COLIDENTIFIER + + See: https://cloud.google.com/bigquery/docs/schemas#column_names + """ + destination_table = f"{dataset_id}.test_to_gbq_w_protected_column_names" + + scalars_df_index = scalars_df_index.rename( + columns={ + "bool_col": "_Table_Suffix", + "bytes_col": "_file_path", + "date_col": "_PARTITIONDATE", + "datetime_col": "_ROW_TIMESTAMP", + "int64_col": "__ROOT__", + "int64_too": "_COLIDENTIFIER", + "numeric_col": "COLIDENTIFIER", # Create a collision at serialization time. + } + )[ + [ + "_Table_Suffix", + "_file_path", + "_PARTITIONDATE", + "_ROW_TIMESTAMP", + "__ROOT__", + "_COLIDENTIFIER", + "COLIDENTIFIER", + ] + ] + scalars_df_index.to_gbq(destination_table, if_exists="replace") + + bf_result = bpd.read_gbq(destination_table, index_col="rowindex").to_pandas() + + # Leading _ characters are removed to make these columns valid in BigQuery. + expected = scalars_pandas_df_index.rename( + columns={ + "bool_col": "Table_Suffix", + "bytes_col": "file_path", + "date_col": "PARTITIONDATE", + "datetime_col": "ROW_TIMESTAMP", + "int64_col": "ROOT__", + "int64_too": "COLIDENTIFIER", + "numeric_col": "COLIDENTIFIER_1", + } + )[ + [ + "Table_Suffix", + "file_path", + "PARTITIONDATE", + "ROW_TIMESTAMP", + "ROOT__", + "COLIDENTIFIER", + "COLIDENTIFIER_1", + ] + ] + + pd.testing.assert_frame_equal(bf_result, expected) + + def test_to_gbq_w_flexible_column_names( scalars_df_index, dataset_id: str, bigquery_client ): From 5c125c99d4632c617425c2ef5c399d17878c0043 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tim=20Swe=C3=B1a=20=28Swast=29?= Date: Tue, 6 May 2025 12:29:28 -0500 Subject: [PATCH 02/36] perf: defer query in `read_gbq` with wildcard tables (#1661) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * perf: defer query in `read_gbq` with wildcard tables * remove obsolete comments * use sql node instead of ibis table node to keep select * from omitting pseudocolumns Fixes this code sample: import bigframes.pandas as bpd df = bpd.read_gbq("bigquery-public-data.google_analytics_sample.ga_sessions_*") df[df["_TABLE_SUFFIX"] == "20161204"].peek() * test with cache and to_gbq * rename columns before caching * remove unnecessary comment * add missing import * do not materialize _TABLE_SUFFIX * fix unit tests * correct number of columns in cache with offsets * fix formatting * 🦉 Updates from OwlBot post-processor See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md * revert datetime change, max_results change * add pseudocolumns to node * fix unit tests * actually fix unit tests * try to rename as part of compile * use correct node for table schema * revert pseudocolumn addition * revert pseudocolumn fix * add test for warning --------- Co-authored-by: Owl Bot --- bigframes/session/_io/bigquery/read_gbq_table.py | 11 ++++++++++- bigframes/session/loader.py | 9 +++------ tests/system/small/test_session.py | 6 +++++- 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/bigframes/session/_io/bigquery/read_gbq_table.py b/bigframes/session/_io/bigquery/read_gbq_table.py index 8d8f247185..6b1cb99c65 100644 --- a/bigframes/session/_io/bigquery/read_gbq_table.py +++ b/bigframes/session/_io/bigquery/read_gbq_table.py @@ -101,7 +101,16 @@ def validate_table( # Anonymous dataset, does not support snapshot ever if table.dataset_id.startswith("_"): pass + # Only true tables support time travel + elif table.table_id.endswith("*"): + msg = bfe.format_message( + "Wildcard tables do not support FOR SYSTEM_TIME AS OF queries. " + "Attempting query without time travel. Be aware that " + "modifications to the underlying data may result in errors or " + "unexpected behavior." + ) + warnings.warn(msg, category=bfe.TimeTravelDisabledWarning) elif table.table_type != "TABLE": if table.table_type == "MATERIALIZED_VIEW": msg = bfe.format_message( @@ -137,7 +146,7 @@ def validate_table( sql_predicate=filter_str, time_travel_timestamp=None, ) - # Any erorrs here should just be raised to user + # Any errors here should just be raised to user bqclient.query_and_wait( snapshot_sql, job_config=bigquery.QueryJobConfig(dry_run=True) ) diff --git a/bigframes/session/loader.py b/bigframes/session/loader.py index f748f0fd76..4924037f89 100644 --- a/bigframes/session/loader.py +++ b/bigframes/session/loader.py @@ -518,11 +518,7 @@ def read_gbq_table( # clustered tables, so fallback to a query. We do this here so that # the index is consistent with tables that have primary keys, even # when max_results is set. - # TODO(b/338419730): We don't need to fallback to a query for wildcard - # tables if we allow some non-determinism when time travel isn't supported. - if max_results is not None or bf_io_bigquery.is_table_with_wildcard_suffix( - table_id - ): + if max_results is not None: # TODO(b/338111344): If we are running a query anyway, we might as # well generate ROW_NUMBER() at the same time. all_columns: Iterable[str] = ( @@ -540,7 +536,7 @@ def read_gbq_table( time_travel_timestamp=None, ) - return self.read_gbq_query( # type: ignore # for dry_run overload + df = self.read_gbq_query( # type: ignore # for dry_run overload query, index_col=index_cols, columns=columns, @@ -548,6 +544,7 @@ def read_gbq_table( use_cache=use_cache, dry_run=dry_run, ) + return df if dry_run: return dry_runs.get_table_stats(table) diff --git a/tests/system/small/test_session.py b/tests/system/small/test_session.py index ad01a95509..ab460d5bc9 100644 --- a/tests/system/small/test_session.py +++ b/tests/system/small/test_session.py @@ -449,11 +449,15 @@ def test_read_gbq_twice_with_same_timestamp(session, penguins_table_id): @pytest.mark.parametrize( "source_table", [ + # Wildcard tables + "bigquery-public-data.noaa_gsod.gsod194*", + # Linked datasets "bigframes-dev.thelook_ecommerce.orders", + # Materialized views "bigframes-dev.bigframes_tests_sys.base_table_mat_view", ], ) -def test_read_gbq_on_linked_dataset_warns(session, source_table): +def test_read_gbq_warns_time_travel_disabled(session, source_table): with warnings.catch_warnings(record=True) as warned: session.read_gbq(source_table, use_cache=False) assert len(warned) == 1 From 67d876076027b6123e49d1d8ddee4e45eaa28f5d Mon Sep 17 00:00:00 2001 From: TrevorBergeron Date: Tue, 6 May 2025 11:34:56 -0700 Subject: [PATCH 03/36] perf: Rechunk result pages client side (#1680) --- bigframes/core/blocks.py | 6 +- bigframes/core/pyarrow_utils.py | 87 ++++++++++++++++++++++ bigframes/session/_io/bigquery/__init__.py | 10 +-- bigframes/session/bq_caching_executor.py | 23 +----- bigframes/session/executor.py | 25 ++++++- bigframes/session/loader.py | 2 - tests/unit/core/test_pyarrow_utils.py | 65 ++++++++++++++++ tests/unit/session/test_io_bigquery.py | 7 +- 8 files changed, 182 insertions(+), 43 deletions(-) create mode 100644 bigframes/core/pyarrow_utils.py create mode 100644 tests/unit/core/test_pyarrow_utils.py diff --git a/bigframes/core/blocks.py b/bigframes/core/blocks.py index 6426b7b22b..d3107a0623 100644 --- a/bigframes/core/blocks.py +++ b/bigframes/core/blocks.py @@ -586,10 +586,10 @@ def to_pandas_batches( self.expr, ordered=True, use_explicit_destination=allow_large_results, - page_size=page_size, - max_results=max_results, ) - for df in execute_result.to_pandas_batches(): + for df in execute_result.to_pandas_batches( + page_size=page_size, max_results=max_results + ): self._copy_index_to_pandas(df) if squeeze: yield df.squeeze(axis=1) diff --git a/bigframes/core/pyarrow_utils.py b/bigframes/core/pyarrow_utils.py new file mode 100644 index 0000000000..eead30d908 --- /dev/null +++ b/bigframes/core/pyarrow_utils.py @@ -0,0 +1,87 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. +from typing import Iterable, Iterator + +import pyarrow as pa + + +class BatchBuffer: + """ + FIFO buffer of pyarrow Record batches + + Not thread-safe. + """ + + def __init__(self): + self._buffer: list[pa.RecordBatch] = [] + self._buffer_size: int = 0 + + def __len__(self): + return self._buffer_size + + def append_batch(self, batch: pa.RecordBatch) -> None: + self._buffer.append(batch) + self._buffer_size += batch.num_rows + + def take_as_batches(self, n: int) -> tuple[pa.RecordBatch, ...]: + if n > len(self): + raise ValueError(f"Cannot take {n} rows, only {len(self)} rows in buffer.") + rows_taken = 0 + sub_batches: list[pa.RecordBatch] = [] + while rows_taken < n: + batch = self._buffer.pop(0) + if batch.num_rows > (n - rows_taken): + sub_batches.append(batch.slice(length=n - rows_taken)) + self._buffer.insert(0, batch.slice(offset=n - rows_taken)) + rows_taken += n - rows_taken + else: + sub_batches.append(batch) + rows_taken += batch.num_rows + + self._buffer_size -= n + return tuple(sub_batches) + + def take_rechunked(self, n: int) -> pa.RecordBatch: + return ( + pa.Table.from_batches(self.take_as_batches(n)) + .combine_chunks() + .to_batches()[0] + ) + + +def chunk_by_row_count( + batches: Iterable[pa.RecordBatch], page_size: int +) -> Iterator[tuple[pa.RecordBatch, ...]]: + buffer = BatchBuffer() + for batch in batches: + buffer.append_batch(batch) + while len(buffer) >= page_size: + yield buffer.take_as_batches(page_size) + + # emit final page, maybe smaller + if len(buffer) > 0: + yield buffer.take_as_batches(len(buffer)) + + +def truncate_pyarrow_iterable( + batches: Iterable[pa.RecordBatch], max_results: int +) -> Iterator[pa.RecordBatch]: + total_yielded = 0 + for batch in batches: + if batch.num_rows >= (max_results - total_yielded): + yield batch.slice(length=max_results - total_yielded) + return + else: + yield batch + total_yielded += batch.num_rows diff --git a/bigframes/session/_io/bigquery/__init__.py b/bigframes/session/_io/bigquery/__init__.py index 6df9424e3b..48268d925d 100644 --- a/bigframes/session/_io/bigquery/__init__.py +++ b/bigframes/session/_io/bigquery/__init__.py @@ -222,8 +222,6 @@ def start_query_with_client( job_config: bigquery.job.QueryJobConfig, location: Optional[str] = None, project: Optional[str] = None, - max_results: Optional[int] = None, - page_size: Optional[int] = None, timeout: Optional[float] = None, api_name: Optional[str] = None, metrics: Optional[bigframes.session.metrics.ExecutionMetrics] = None, @@ -244,8 +242,6 @@ def start_query_with_client( location=location, project=project, api_timeout=timeout, - page_size=page_size, - max_results=max_results, ) if metrics is not None: metrics.count_job_stats(row_iterator=results_iterator) @@ -267,14 +263,10 @@ def start_query_with_client( if opts.progress_bar is not None and not query_job.configuration.dry_run: results_iterator = formatting_helpers.wait_for_query_job( query_job, - max_results=max_results, progress_bar=opts.progress_bar, - page_size=page_size, ) else: - results_iterator = query_job.result( - max_results=max_results, page_size=page_size - ) + results_iterator = query_job.result() if metrics is not None: metrics.count_job_stats(query_job=query_job) diff --git a/bigframes/session/bq_caching_executor.py b/bigframes/session/bq_caching_executor.py index 4c10d76253..533f49b666 100644 --- a/bigframes/session/bq_caching_executor.py +++ b/bigframes/session/bq_caching_executor.py @@ -106,8 +106,6 @@ def execute( *, ordered: bool = True, use_explicit_destination: Optional[bool] = None, - page_size: Optional[int] = None, - max_results: Optional[int] = None, ) -> executor.ExecuteResult: if use_explicit_destination is None: use_explicit_destination = bigframes.options.bigquery.allow_large_results @@ -127,8 +125,6 @@ def execute( return self._execute_plan( plan, ordered=ordered, - page_size=page_size, - max_results=max_results, destination=destination_table, ) @@ -281,8 +277,6 @@ def _run_execute_query( sql: str, job_config: Optional[bq_job.QueryJobConfig] = None, api_name: Optional[str] = None, - page_size: Optional[int] = None, - max_results: Optional[int] = None, query_with_job: bool = True, ) -> Tuple[bq_table.RowIterator, Optional[bigquery.QueryJob]]: """ @@ -303,8 +297,6 @@ def _run_execute_query( sql, job_config=job_config, api_name=api_name, - max_results=max_results, - page_size=page_size, metrics=self.metrics, query_with_job=query_with_job, ) @@ -479,16 +471,13 @@ def _execute_plan( self, plan: nodes.BigFrameNode, ordered: bool, - page_size: Optional[int] = None, - max_results: Optional[int] = None, destination: Optional[bq_table.TableReference] = None, peek: Optional[int] = None, ): """Just execute whatever plan as is, without further caching or decomposition.""" # First try to execute fast-paths - # TODO: Allow page_size and max_results by rechunking/truncating results - if (not page_size) and (not max_results) and (not destination) and (not peek): + if (not destination) and (not peek): for semi_executor in self._semi_executors: maybe_result = semi_executor.execute(plan, ordered=ordered) if maybe_result: @@ -504,20 +493,12 @@ def _execute_plan( iterator, query_job = self._run_execute_query( sql=sql, job_config=job_config, - page_size=page_size, - max_results=max_results, query_with_job=(destination is not None), ) # Though we provide the read client, iterator may or may not use it based on what is efficient for the result def iterator_supplier(): - # Workaround issue fixed by: https://github.com/googleapis/python-bigquery/pull/2154 - if iterator._page_size is not None or iterator.max_results is not None: - return iterator.to_arrow_iterable(bqstorage_client=None) - else: - return iterator.to_arrow_iterable( - bqstorage_client=self.bqstoragereadclient - ) + return iterator.to_arrow_iterable(bqstorage_client=self.bqstoragereadclient) if query_job: size_bytes = self.bqclient.get_table(query_job.destination).num_bytes diff --git a/bigframes/session/executor.py b/bigframes/session/executor.py index 9075f4eee6..fd9d0a083f 100644 --- a/bigframes/session/executor.py +++ b/bigframes/session/executor.py @@ -25,6 +25,7 @@ import pyarrow import bigframes.core +from bigframes.core import pyarrow_utils import bigframes.core.schema import bigframes.session._io.pandas as io_pandas @@ -55,10 +56,28 @@ def to_arrow_table(self) -> pyarrow.Table: def to_pandas(self) -> pd.DataFrame: return io_pandas.arrow_to_pandas(self.to_arrow_table(), self.schema) - def to_pandas_batches(self) -> Iterator[pd.DataFrame]: + def to_pandas_batches( + self, page_size: Optional[int] = None, max_results: Optional[int] = None + ) -> Iterator[pd.DataFrame]: + assert (page_size is None) or (page_size > 0) + assert (max_results is None) or (max_results > 0) + batch_iter: Iterator[ + Union[pyarrow.Table, pyarrow.RecordBatch] + ] = self.arrow_batches() + if max_results is not None: + batch_iter = pyarrow_utils.truncate_pyarrow_iterable( + batch_iter, max_results + ) + + if page_size is not None: + batches_iter = pyarrow_utils.chunk_by_row_count(batch_iter, page_size) + batch_iter = map( + lambda batches: pyarrow.Table.from_batches(batches), batches_iter + ) + yield from map( functools.partial(io_pandas.arrow_to_pandas, schema=self.schema), - self.arrow_batches(), + batch_iter, ) def to_py_scalar(self): @@ -107,8 +126,6 @@ def execute( *, ordered: bool = True, use_explicit_destination: Optional[bool] = False, - page_size: Optional[int] = None, - max_results: Optional[int] = None, ) -> ExecuteResult: """ Execute the ArrayValue, storing the result to a temporary session-owned table. diff --git a/bigframes/session/loader.py b/bigframes/session/loader.py index 4924037f89..d9adb8683a 100644 --- a/bigframes/session/loader.py +++ b/bigframes/session/loader.py @@ -906,7 +906,6 @@ def _start_query( self, sql: str, job_config: Optional[google.cloud.bigquery.QueryJobConfig] = None, - max_results: Optional[int] = None, timeout: Optional[float] = None, api_name: Optional[str] = None, ) -> Tuple[google.cloud.bigquery.table.RowIterator, bigquery.QueryJob]: @@ -925,7 +924,6 @@ def _start_query( self._bqclient, sql, job_config=job_config, - max_results=max_results, timeout=timeout, api_name=api_name, ) diff --git a/tests/unit/core/test_pyarrow_utils.py b/tests/unit/core/test_pyarrow_utils.py new file mode 100644 index 0000000000..155c36d268 --- /dev/null +++ b/tests/unit/core/test_pyarrow_utils.py @@ -0,0 +1,65 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +import itertools + +import numpy as np +import pyarrow as pa +import pytest + +from bigframes.core import pyarrow_utils + +PA_TABLE = pa.table({f"col_{i}": np.random.rand(1000) for i in range(10)}) + +# 17, 3, 929 coprime +N = 17 +MANY_SMALL_BATCHES = PA_TABLE.to_batches(max_chunksize=3) +FEW_BIG_BATCHES = PA_TABLE.to_batches(max_chunksize=929) + + +@pytest.mark.parametrize( + ["batches", "page_size"], + [ + (MANY_SMALL_BATCHES, N), + (FEW_BIG_BATCHES, N), + ], +) +def test_chunk_by_row_count(batches, page_size): + results = list(pyarrow_utils.chunk_by_row_count(batches, page_size=page_size)) + + for i, batches in enumerate(results): + if i != len(results) - 1: + assert sum(map(lambda x: x.num_rows, batches)) == page_size + else: + # final page can be smaller + assert sum(map(lambda x: x.num_rows, batches)) <= page_size + + reconstructed = pa.Table.from_batches(itertools.chain.from_iterable(results)) + assert reconstructed.equals(PA_TABLE) + + +@pytest.mark.parametrize( + ["batches", "max_rows"], + [ + (MANY_SMALL_BATCHES, N), + (FEW_BIG_BATCHES, N), + ], +) +def test_truncate_pyarrow_iterable(batches, max_rows): + results = list( + pyarrow_utils.truncate_pyarrow_iterable(batches, max_results=max_rows) + ) + + reconstructed = pa.Table.from_batches(results) + assert reconstructed.equals(PA_TABLE.slice(length=max_rows)) diff --git a/tests/unit/session/test_io_bigquery.py b/tests/unit/session/test_io_bigquery.py index af2c7714ab..14e5d1c2fe 100644 --- a/tests/unit/session/test_io_bigquery.py +++ b/tests/unit/session/test_io_bigquery.py @@ -199,11 +199,11 @@ def test_add_and_trim_labels_length_limit_met(): @pytest.mark.parametrize( - ("max_results", "timeout", "api_name"), - [(None, None, None), (100, 30.0, "test_api")], + ("timeout", "api_name"), + [(None, None), (30.0, "test_api")], ) def test_start_query_with_client_labels_length_limit_met( - mock_bq_client, max_results, timeout, api_name + mock_bq_client, timeout, api_name ): sql = "select * from abc" cur_labels = { @@ -230,7 +230,6 @@ def test_start_query_with_client_labels_length_limit_met( mock_bq_client, sql, job_config, - max_results=max_results, timeout=timeout, api_name=api_name, ) From 9d4a59ddf22793d4e0587ea2f8648fae937875f3 Mon Sep 17 00:00:00 2001 From: Shenyang Cai Date: Tue, 6 May 2025 13:06:49 -0700 Subject: [PATCH 04/36] feat: Add "dayofyear" property for `dt` accessors (#1692) * implement dayofyear. tests to follow * add tests * add Python doc --- bigframes/core/compile/scalar_op_compiler.py | 7 +++++ bigframes/operations/__init__.py | 2 ++ bigframes/operations/date_ops.py | 5 ++++ bigframes/operations/datetimes.py | 4 +++ .../system/small/operations/test_datetimes.py | 14 +++++++++ .../pandas/core/indexes/accessor.py | 29 +++++++++++++++++++ 6 files changed, 61 insertions(+) diff --git a/bigframes/core/compile/scalar_op_compiler.py b/bigframes/core/compile/scalar_op_compiler.py index 8243627a91..2b9208137b 100644 --- a/bigframes/core/compile/scalar_op_compiler.py +++ b/bigframes/core/compile/scalar_op_compiler.py @@ -676,6 +676,13 @@ def dayofweek_op_impl(x: ibis_types.Value): ) +@scalar_op_compiler.register_unary_op(ops.dayofyear_op) +def dayofyear_op_impl(x: ibis_types.Value): + return ( + typing.cast(ibis_types.TimestampValue, x).day_of_year().cast(ibis_dtypes.int64) + ) + + @scalar_op_compiler.register_unary_op(ops.hour_op) def hour_op_impl(x: ibis_types.Value): return typing.cast(ibis_types.TimestampValue, x).hour().cast(ibis_dtypes.int64) diff --git a/bigframes/operations/__init__.py b/bigframes/operations/__init__.py index 74ff5c0f98..c8ccaf2a25 100644 --- a/bigframes/operations/__init__.py +++ b/bigframes/operations/__init__.py @@ -42,6 +42,7 @@ date_diff_op, day_op, dayofweek_op, + dayofyear_op, month_op, quarter_op, year_op, @@ -261,6 +262,7 @@ "month_op", "year_op", "dayofweek_op", + "dayofyear_op", "quarter_op", # Time ops "hour_op", diff --git a/bigframes/operations/date_ops.py b/bigframes/operations/date_ops.py index 32d8eec118..9bcdddb8df 100644 --- a/bigframes/operations/date_ops.py +++ b/bigframes/operations/date_ops.py @@ -39,6 +39,11 @@ type_signature=op_typing.DATELIKE_ACCESSOR, ) +dayofyear_op = base_ops.create_unary_op( + name="dayofyear", + type_signature=op_typing.DATELIKE_ACCESSOR, +) + quarter_op = base_ops.create_unary_op( name="quarter", type_signature=op_typing.DATELIKE_ACCESSOR, diff --git a/bigframes/operations/datetimes.py b/bigframes/operations/datetimes.py index 7d25ac3622..032bd50390 100644 --- a/bigframes/operations/datetimes.py +++ b/bigframes/operations/datetimes.py @@ -43,6 +43,10 @@ def day(self) -> series.Series: def dayofweek(self) -> series.Series: return self._apply_unary_op(ops.dayofweek_op) + @property + def dayofyear(self) -> series.Series: + return self._apply_unary_op(ops.dayofyear_op) + @property def date(self) -> series.Series: return self._apply_unary_op(ops.date_op) diff --git a/tests/system/small/operations/test_datetimes.py b/tests/system/small/operations/test_datetimes.py index 0463124309..705439fd96 100644 --- a/tests/system/small/operations/test_datetimes.py +++ b/tests/system/small/operations/test_datetimes.py @@ -81,6 +81,20 @@ def test_dt_dayofweek(scalars_dfs, col_name): assert_series_equal(pd_result, bf_result, check_dtype=False) +@pytest.mark.parametrize( + ("col_name",), + DATE_COLUMNS, +) +def test_dt_dayofyear(scalars_dfs, col_name): + pytest.importorskip("pandas", minversion="2.0.0") + scalars_df, scalars_pandas_df = scalars_dfs + bf_series: bigframes.series.Series = scalars_df[col_name] + bf_result = bf_series.dt.dayofyear.to_pandas() + pd_result = scalars_pandas_df[col_name].dt.dayofyear + + assert_series_equal(pd_result, bf_result, check_dtype=False) + + @pytest.mark.parametrize( ("col_name",), DATETIME_COL_NAMES, diff --git a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py index f34612cb11..a3c0d59e46 100644 --- a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py +++ b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py @@ -66,6 +66,35 @@ def dayofweek(self): raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + @property + def dayofyear(self): + """The ordinal day of the year. + + **Examples:** + + >>> import pandas as pd + >>> import bigframes.pandas as bpd + >>> bpd.options.display.progress_bar = None + >>> s = bpd.Series( + ... pd.date_range('2016-12-28', '2017-01-03', freq='D').to_series() + ... ) + >>> s.dt.dayofyear + 2016-12-28 00:00:00 363 + 2016-12-29 00:00:00 364 + 2016-12-30 00:00:00 365 + 2016-12-31 00:00:00 366 + 2017-01-01 00:00:00 1 + 2017-01-02 00:00:00 2 + 2017-01-03 00:00:00 3 + dtype: Int64 + dtype: Int64 + + Returns: + Series: Containing integers indicating the day number. + """ + + raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + @property def date(self): """Returns a Series with the date part of Timestamps without time and From 46a9c53256be2a293f96122ba6b330564383bcd5 Mon Sep 17 00:00:00 2001 From: Chelsea Lin Date: Tue, 6 May 2025 16:19:56 -0700 Subject: [PATCH 05/36] feat: add bigframes.bigquery.json_value (#1697) --- bigframes/bigquery/__init__.py | 2 ++ bigframes/bigquery/_operations/json.py | 34 ++++++++++++++++++++++++ tests/system/small/bigquery/test_json.py | 28 +++++++++++++++++++ 3 files changed, 64 insertions(+) diff --git a/bigframes/bigquery/__init__.py b/bigframes/bigquery/__init__.py index 6eb725975e..02dd77fdd9 100644 --- a/bigframes/bigquery/__init__.py +++ b/bigframes/bigquery/__init__.py @@ -38,6 +38,7 @@ json_extract_array, json_extract_string_array, json_set, + json_value, parse_json, ) from bigframes.bigquery._operations.search import create_vector_index, vector_search @@ -61,6 +62,7 @@ "json_extract", "json_extract_array", "json_extract_string_array", + "json_value", "parse_json", # search ops "create_vector_index", diff --git a/bigframes/bigquery/_operations/json.py b/bigframes/bigquery/_operations/json.py index 07efc5fa51..b59fe40d99 100644 --- a/bigframes/bigquery/_operations/json.py +++ b/bigframes/bigquery/_operations/json.py @@ -231,6 +231,40 @@ def json_extract_string_array( return array_series +def json_value( + input: series.Series, + json_path: str, +) -> series.Series: + """Extracts a JSON scalar value and converts it to a SQL ``STRING`` value. In + addtion, this function: + - Removes the outermost quotes and unescapes the values. + - Returns a SQL ``NULL`` if a non-scalar value is selected. + - Uses double quotes to escape invalid ``JSON_PATH`` characters in JSON keys. + + **Examples:** + + >>> import bigframes.pandas as bpd + >>> import bigframes.bigquery as bbq + >>> bpd.options.display.progress_bar = None + + >>> s = bpd.Series(['{"name": "Jakob", "age": "6"}', '{"name": "Jakob", "age": []}']) + >>> bbq.json_value(s, json_path="$.age") + 0 6 + 1 + dtype: string + + Args: + input (bigframes.series.Series): + The Series containing JSON data (as native JSON objects or JSON-formatted strings). + json_path (str): + The JSON path identifying the data that you want to obtain from the input. + + Returns: + bigframes.series.Series: A new Series with the JSON-formatted STRING. + """ + return input._apply_unary_op(ops.JSONValue(json_path=json_path)) + + @utils.preview(name="The JSON-related API `parse_json`") def parse_json( input: series.Series, diff --git a/tests/system/small/bigquery/test_json.py b/tests/system/small/bigquery/test_json.py index df5a524b55..b9d21f226a 100644 --- a/tests/system/small/bigquery/test_json.py +++ b/tests/system/small/bigquery/test_json.py @@ -212,6 +212,34 @@ def test_json_extract_string_array_w_invalid_series_type(): bbq.json_extract_string_array(s) +def test_json_value_from_json(): + s = bpd.Series( + ['{"a": {"b": [1, 2]}}', '{"a": {"c": 1}}', '{"a": {"b": 0}}'], + dtype=dtypes.JSON_DTYPE, + ) + actual = bbq.json_value(s, "$.a.b") + expected = bpd.Series([None, None, "0"], dtype=dtypes.STRING_DTYPE) + + pd.testing.assert_series_equal(actual.to_pandas(), expected.to_pandas()) + + +def test_json_value_from_string(): + s = bpd.Series( + ['{"a": {"b": [1, 2]}}', '{"a": {"c": 1}}', '{"a": {"b": 0}}'], + dtype=pd.StringDtype(storage="pyarrow"), + ) + actual = bbq.json_value(s, "$.a.b") + expected = bpd.Series([None, None, "0"], dtype=dtypes.STRING_DTYPE) + + pd.testing.assert_series_equal(actual.to_pandas(), expected.to_pandas()) + + +def test_json_value_w_invalid_series_type(): + s = bpd.Series([1, 2]) + with pytest.raises(TypeError): + bbq.json_value(s, "$.a") + + def test_parse_json_w_invalid_series_type(): s = bpd.Series([1, 2]) with pytest.raises(TypeError): From e4c85ba4813469d39edd7352201aefc26642d14c Mon Sep 17 00:00:00 2001 From: Garrett Wu <6505921+GarrettWu@users.noreply.github.com> Date: Tue, 6 May 2025 18:12:50 -0700 Subject: [PATCH 06/36] feat: publish bigframes blob(Multimodal) to preview (#1693) * feat: publish bigframes blob(Multimodal) to preview * fix * fix * reset df notebook --- bigframes/_config/display_options.py | 4 + bigframes/_config/experiment_options.py | 64 +++-- bigframes/dataframe.py | 24 +- bigframes/ml/llm.py | 15 +- bigframes/operations/blob.py | 73 +---- bigframes/operations/strings.py | 8 +- bigframes/session/__init__.py | 16 +- .../experimental/multimodal_dataframe.ipynb | 253 +++++++++--------- samples/snippets/multimodal_test.py | 2 - tests/system/conftest.py | 4 - tests/system/large/blob/test_function.py | 22 -- tests/system/small/blob/test_io.py | 6 - tests/system/small/blob/test_properties.py | 58 ++-- tests/system/small/blob/test_urls.py | 5 - tests/system/small/ml/test_llm.py | 4 - tests/system/small/ml/test_multimodal_llm.py | 7 - tests/unit/_config/test_experiment_options.py | 15 -- .../pandas/core/config_init.py | 6 + 18 files changed, 238 insertions(+), 348 deletions(-) diff --git a/bigframes/_config/display_options.py b/bigframes/_config/display_options.py index 2af07d30a8..dc8ab34f2a 100644 --- a/bigframes/_config/display_options.py +++ b/bigframes/_config/display_options.py @@ -35,6 +35,10 @@ class DisplayOptions: max_info_rows: Optional[int] = 200000 memory_usage: bool = True + blob_display: bool = True + blob_display_width: Optional[int] = None + blob_display_height: Optional[int] = None + @contextlib.contextmanager def pandas_repr(display_options: DisplayOptions): diff --git a/bigframes/_config/experiment_options.py b/bigframes/_config/experiment_options.py index cfbcc04cfd..024de392c0 100644 --- a/bigframes/_config/experiment_options.py +++ b/bigframes/_config/experiment_options.py @@ -15,6 +15,7 @@ from typing import Optional import warnings +import bigframes import bigframes.exceptions as bfe @@ -26,10 +27,6 @@ class ExperimentOptions: def __init__(self): self._semantic_operators: bool = False self._ai_operators: bool = False - self._blob: bool = False - self._blob_display: bool = True - self._blob_display_width: Optional[int] = None - self._blob_display_height: Optional[int] = None @property def semantic_operators(self) -> bool: @@ -60,41 +57,72 @@ def ai_operators(self, value: bool): @property def blob(self) -> bool: - return self._blob + msg = bfe.format_message( + "BigFrames Blob is in preview now. This flag is no longer needed." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) + return True @blob.setter def blob(self, value: bool): - if value is True: - msg = bfe.format_message( - "BigFrames Blob is still under experiments. It may not work and " - "subject to change in the future." - ) - warnings.warn(msg, category=bfe.PreviewWarning) - self._blob = value + msg = bfe.format_message( + "BigFrames Blob is in preview now. This flag is no longer needed." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) @property def blob_display(self) -> bool: """Whether to display the blob content in notebook DataFrame preview. Default True.""" - return self._blob_display + msg = bfe.format_message( + "BigFrames Blob is in preview now. The option has been moved to bigframes.options.display.blob_display." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) + + return bigframes.options.display.blob_display @blob_display.setter def blob_display(self, value: bool): - self._blob_display = value + msg = bfe.format_message( + "BigFrames Blob is in preview now. The option has been moved to bigframes.options.display.blob_display." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) + + bigframes.options.display.blob_display = value @property def blob_display_width(self) -> Optional[int]: """Width in pixels that the blob constrained to.""" - return self._blob_display_width + msg = bfe.format_message( + "BigFrames Blob is in preview now. The option has been moved to bigframes.options.display.blob_display_width." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) + + return bigframes.options.display.blob_display_width @blob_display_width.setter def blob_display_width(self, value: Optional[int]): - self._blob_display_width = value + msg = bfe.format_message( + "BigFrames Blob is in preview now. The option has been moved to bigframes.options.display.blob_display_width." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) + + bigframes.options.display.blob_display_width = value @property def blob_display_height(self) -> Optional[int]: """Height in pixels that the blob constrained to.""" - return self._blob_display_height + msg = bfe.format_message( + "BigFrames Blob is in preview now. The option has been moved to bigframes.options.display.blob_display_height." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) + + return bigframes.options.display.blob_display_height @blob_display_height.setter def blob_display_height(self, value: Optional[int]): - self._blob_display_height = value + msg = bfe.format_message( + "BigFrames Blob is in preview now. The option has been moved to bigframes.options.display.blob_display_height." + ) + warnings.warn(msg, category=bfe.ApiDeprecationWarning) + + bigframes.options.display.blob_display_height = value diff --git a/bigframes/dataframe.py b/bigframes/dataframe.py index 9cb388329e..e64aec307a 100644 --- a/bigframes/dataframe.py +++ b/bigframes/dataframe.py @@ -768,14 +768,11 @@ def _repr_html_(self) -> str: return formatter.repr_query_job(self._compute_dry_run()) df = self.copy() - if ( - bigframes.options.experiments.blob - and bigframes.options.experiments.blob_display - ): + if bigframes.options.display.blob_display: blob_cols = [ - col - for col in df.columns - if df[col].dtype == bigframes.dtypes.OBJ_REF_DTYPE + series_name + for series_name, series in df.items() + if series.dtype == bigframes.dtypes.OBJ_REF_DTYPE ] for col in blob_cols: # TODO(garrettwu): Not necessary to get access urls for all the rows. Update when having a to get URLs from local data. @@ -794,10 +791,7 @@ def _repr_html_(self) -> str: with display_options.pandas_repr(opts): # Allows to preview images in the DataFrame. The implementation changes the string repr as well, that it doesn't truncate strings or escape html charaters such as "<" and ">". We may need to implement a full-fledged repr module to better support types not in pandas. - if ( - bigframes.options.experiments.blob - and bigframes.options.experiments.blob_display - ): + if bigframes.options.display.blob_display and blob_cols: def obj_ref_rt_to_html(obj_ref_rt) -> str: obj_ref_rt_json = json.loads(obj_ref_rt) @@ -809,12 +803,12 @@ def obj_ref_rt_to_html(obj_ref_rt) -> str: ) if content_type.startswith("image"): size_str = "" - if bigframes.options.experiments.blob_display_width: - size_str = f' width="{bigframes.options.experiments.blob_display_width}"' - if bigframes.options.experiments.blob_display_height: + if bigframes.options.display.blob_display_width: + size_str = f' width="{bigframes.options.display.blob_display_width}"' + if bigframes.options.display.blob_display_height: size_str = ( size_str - + f' height="{bigframes.options.experiments.blob_display_height}"' + + f' height="{bigframes.options.display.blob_display_height}"' ) url = obj_ref_rt_json["access_urls"]["read_url"] return f'' diff --git a/bigframes/ml/llm.py b/bigframes/ml/llm.py index bd414102e1..111ad20f8a 100644 --- a/bigframes/ml/llm.py +++ b/bigframes/ml/llm.py @@ -250,7 +250,10 @@ class MultimodalEmbeddingGenerator(base.RetriableRemotePredictor): """Multimodal embedding generator LLM model. .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. + BigFrames Blob is subject to the "Pre-GA Offerings Terms" in the General Service Terms section of the + Service Specific Terms(https://cloud.google.com/terms/service-terms#1). Pre-GA products and features are available "as is" + and might have limited support. For more information, see the launch stage descriptions + (https://cloud.google.com/products#product-launch-stages). Args: model_name (str, Default to "multimodalembedding@001"): @@ -271,8 +274,6 @@ def __init__( session: Optional[bigframes.Session] = None, connection_name: Optional[str] = None, ): - if not bigframes.options.experiments.blob: - raise NotImplementedError() if model_name is None: model_name = "multimodalembedding@001" msg = exceptions.format_message(_REMOVE_DEFAULT_MODEL_WARNING) @@ -610,7 +611,10 @@ def predict( prompt (Iterable of str or bigframes.series.Series, or None, default None): .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. + BigFrames Blob is subject to the "Pre-GA Offerings Terms" in the General Service Terms section of the + Service Specific Terms(https://cloud.google.com/terms/service-terms#1). Pre-GA products and features are available "as is" + and might have limited support. For more information, see the launch stage descriptions + (https://cloud.google.com/products#product-launch-stages). Construct a prompt struct column for prediction based on the input. The input must be an Iterable that can take string literals, such as "summarize", string column(s) of X, such as X["str_col"], or blob column(s) of X, such as X["blob_col"]. @@ -646,9 +650,6 @@ def predict( (X,) = utils.batch_convert_to_dataframe(X, session=session) if prompt: - if not bigframes.options.experiments.blob: - raise NotImplementedError() - if self.model_name not in _GEMINI_MULTIMODAL_ENDPOINTS: raise NotImplementedError( f"GeminiTextGenerator only supports model_name {', '.join(_GEMINI_MULTIMODAL_ENDPOINTS)} for Multimodal prompt." diff --git a/bigframes/operations/blob.py b/bigframes/operations/blob.py index 2c6e5fca7f..d211c2b918 100644 --- a/bigframes/operations/blob.py +++ b/bigframes/operations/blob.py @@ -40,21 +40,18 @@ class BlobAccessor(base.SeriesMethods): Blob functions for Series and Index. .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. + BigFrames Blob is subject to the "Pre-GA Offerings Terms" in the General Service Terms section of the + Service Specific Terms(https://cloud.google.com/terms/service-terms#1). Pre-GA products and features are available "as is" + and might have limited support. For more information, see the launch stage descriptions + (https://cloud.google.com/products#product-launch-stages). """ def __init__(self, *args, **kwargs): - if not bigframes.options.experiments.blob: - raise NotImplementedError() - super().__init__(*args, **kwargs) def uri(self) -> bigframes.series.Series: """URIs of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: URIs as string.""" s = bigframes.series.Series(self._block) @@ -64,9 +61,6 @@ def uri(self) -> bigframes.series.Series: def authorizer(self) -> bigframes.series.Series: """Authorizers of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: Autorithers(connection) as string.""" s = bigframes.series.Series(self._block) @@ -76,9 +70,6 @@ def authorizer(self) -> bigframes.series.Series: def version(self) -> bigframes.series.Series: """Versions of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: Version as string.""" # version must be retrieved after fetching metadata @@ -87,9 +78,6 @@ def version(self) -> bigframes.series.Series: def metadata(self) -> bigframes.series.Series: """Retrieve the metadata of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: JSON metadata of the Blob. Contains fields: content_type, md5_hash, size and updated(time).""" details_json = self._apply_unary_op(ops.obj_fetch_metadata_op).struct.field( @@ -102,9 +90,6 @@ def metadata(self) -> bigframes.series.Series: def content_type(self) -> bigframes.series.Series: """Retrieve the content type of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: string of the content type.""" return ( @@ -116,9 +101,6 @@ def content_type(self) -> bigframes.series.Series: def md5_hash(self) -> bigframes.series.Series: """Retrieve the md5 hash of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: string of the md5 hash.""" return ( @@ -130,9 +112,6 @@ def md5_hash(self) -> bigframes.series.Series: def size(self) -> bigframes.series.Series: """Retrieve the file size of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: file size in bytes.""" return ( @@ -145,9 +124,6 @@ def size(self) -> bigframes.series.Series: def updated(self) -> bigframes.series.Series: """Retrieve the updated time of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: updated time as UTC datetime.""" import bigframes.pandas as bpd @@ -204,9 +180,6 @@ def _df_apply_udf( def read_url(self) -> bigframes.series.Series: """Retrieve the read URL of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: Read only URLs.""" return self._get_runtime(mode="R")._apply_unary_op( @@ -216,9 +189,6 @@ def read_url(self) -> bigframes.series.Series: def write_url(self) -> bigframes.series.Series: """Retrieve the write URL of the Blob. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Returns: bigframes.series.Series: Writable URLs.""" return self._get_runtime(mode="RW")._apply_unary_op( @@ -235,17 +205,14 @@ def display( ): """Display the blob content in the IPython Notebook environment. Only works for image type now. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Args: n (int, default 3): number of sample blob objects to display. content_type (str, default ""): content type of the blob. If unset, use the blob metadata of the storage. Possible values are "image", "audio" and "video". - width (int or None, default None): width in pixels that the image/video are constrained to. If unset, use the global setting in bigframes.options.experiments.blob_display_width, otherwise image/video's original size or ratio is used. No-op for other content types. - height (int or None, default None): height in pixels that the image/video are constrained to. If unset, use the global setting in bigframes.options.experiments.blob_display_height, otherwise image/video's original size or ratio is used. No-op for other content types. + width (int or None, default None): width in pixels that the image/video are constrained to. If unset, use the global setting in bigframes.options.display.blob_display_width, otherwise image/video's original size or ratio is used. No-op for other content types. + height (int or None, default None): height in pixels that the image/video are constrained to. If unset, use the global setting in bigframes.options.display.blob_display_height, otherwise image/video's original size or ratio is used. No-op for other content types. """ - width = width or bigframes.options.experiments.blob_display_width - height = height or bigframes.options.experiments.blob_display_height + width = width or bigframes.options.display.blob_display_width + height = height or bigframes.options.display.blob_display_height # col name doesn't matter here. Rename to avoid column name conflicts df = bigframes.series.Series(self._block).rename("blob_col").to_frame() @@ -296,10 +263,6 @@ def session(self): def _resolve_connection(self, connection: Optional[str] = None) -> str: """Resovle the BigQuery connection. - .. note:: - BigFrames Blob is still under experiments. It may not work and - subject to change in the future. - Args: connection (str or None, default None): BQ connection used for function internet transactions, and the output blob if "dst" is @@ -324,10 +287,6 @@ def get_runtime_json_str( ) -> bigframes.series.Series: """Get the runtime (contains signed URL to access gcs data) and apply the ToJSONSTring transformation. - .. note:: - BigFrames Blob is still under experiments. It may not work and - subject to change in the future. - Args: mode(str or str, default "R"): the mode for accessing the runtime. Default to "R". Possible values are "R" (read-only) and @@ -353,9 +312,6 @@ def image_blur( ) -> bigframes.series.Series: """Blurs images. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Args: ksize (tuple(int, int)): Kernel size. dst (str or bigframes.series.Series or None, default None): Output destination. Can be one of: @@ -438,9 +394,6 @@ def image_resize( ): """Resize images. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Args: dsize (tuple(int, int), default (0, 0)): Destination size. If set to 0, fx and fy parameters determine the size. fx (float, default 0.0): scale factor along the horizontal axis. If set to 0.0, dsize parameter determines the output size. @@ -534,9 +487,6 @@ def image_normalize( ) -> bigframes.series.Series: """Normalize images. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Args: alpha (float, default 1.0): Norm value to normalize to or the lower range boundary in case of the range normalization. beta (float, default 0.0): Upper range boundary in case of the range normalization; it is not used for the norm normalization. @@ -622,10 +572,6 @@ def pdf_extract( ) -> bigframes.series.Series: """Extracts text from PDF URLs and saves the text as string. - .. note:: - BigFrames Blob is still under experiments. It may not work and - subject to change in the future. - Args: connection (str or None, default None): BQ connection used for function internet transactions, and the output blob if "dst" @@ -688,9 +634,6 @@ def pdf_chunk( """Extracts and chunks text from PDF URLs and saves the text as arrays of strings. - .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. - Args: connection (str or None, default None): BQ connection used for function internet transactions, and the output blob if "dst" diff --git a/bigframes/operations/strings.py b/bigframes/operations/strings.py index 784af8418d..529dd87797 100644 --- a/bigframes/operations/strings.py +++ b/bigframes/operations/strings.py @@ -288,7 +288,10 @@ def to_blob(self, connection: Optional[str] = None) -> series.Series: """Create a BigFrames Blob series from a series of URIs. .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. + BigFrames Blob is subject to the "Pre-GA Offerings Terms" in the General Service Terms section of the + Service Specific Terms(https://cloud.google.com/terms/service-terms#1). Pre-GA products and features are available "as is" + and might have limited support. For more information, see the launch stage descriptions + (https://cloud.google.com/products#product-launch-stages). Args: @@ -301,9 +304,6 @@ def to_blob(self, connection: Optional[str] = None) -> series.Series: bigframes.series.Series: Blob Series. """ - if not bigframes.options.experiments.blob: - raise NotImplementedError() - session = self._block.session connection = session._create_bq_connection(connection=connection) return self._apply_binary_op(connection, ops.obj_make_ref_op) diff --git a/bigframes/session/__init__.py b/bigframes/session/__init__.py index 998e6e57bc..1081270c76 100644 --- a/bigframes/session/__init__.py +++ b/bigframes/session/__init__.py @@ -1905,7 +1905,10 @@ def from_glob_path( If you have an existing BQ Object Table, use read_gbq_object_table(). .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. + BigFrames Blob is subject to the "Pre-GA Offerings Terms" in the General Service Terms section of the + Service Specific Terms(https://cloud.google.com/terms/service-terms#1). Pre-GA products and features are available "as is" + and might have limited support. For more information, see the launch stage descriptions + (https://cloud.google.com/products#product-launch-stages). Args: path (str): @@ -1920,9 +1923,6 @@ def from_glob_path( bigframes.pandas.DataFrame: Result BigFrames DataFrame. """ - if not bigframes.options.experiments.blob: - raise NotImplementedError() - # TODO(garrettwu): switch to pseudocolumn when b/374988109 is done. connection = self._create_bq_connection(connection=connection) @@ -1966,7 +1966,10 @@ def read_gbq_object_table( This function dosen't retrieve the object table data. If you want to read the data, use read_gbq() instead. .. note:: - BigFrames Blob is still under experiments. It may not work and subject to change in the future. + BigFrames Blob is subject to the "Pre-GA Offerings Terms" in the General Service Terms section of the + Service Specific Terms(https://cloud.google.com/terms/service-terms#1). Pre-GA products and features are available "as is" + and might have limited support. For more information, see the launch stage descriptions + (https://cloud.google.com/products#product-launch-stages). Args: object_table (str): name of the object table of form ... @@ -1976,9 +1979,6 @@ def read_gbq_object_table( bigframes.pandas.DataFrame: Result BigFrames DataFrame. """ - if not bigframes.options.experiments.blob: - raise NotImplementedError() - # TODO(garrettwu): switch to pseudocolumn when b/374988109 is done. table = self.bqclient.get_table(object_table) connection = table._properties["externalDataConfiguration"]["connectionId"] diff --git a/notebooks/experimental/multimodal_dataframe.ipynb b/notebooks/experimental/multimodal_dataframe.ipynb index 9c76654a53..4a0cd57a45 100644 --- a/notebooks/experimental/multimodal_dataframe.ipynb +++ b/notebooks/experimental/multimodal_dataframe.ipynb @@ -2,7 +2,7 @@ "cells": [ { "cell_type": "code", - "execution_count": null, + "execution_count": 1, "metadata": {}, "outputs": [], "source": [ @@ -74,7 +74,7 @@ }, { "cell_type": "code", - "execution_count": 1, + "execution_count": 2, "metadata": { "colab": { "base_uri": "https://localhost:8080/" @@ -82,17 +82,7 @@ "id": "bGyhLnfEeB0X", "outputId": "83ac8b64-3f44-4d43-d089-28a5026cbb42" }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/_config/experiment_options.py:68: PreviewWarning: BigFrames Blob is still under experiments. It may not work and subject\n", - "to change in the future.\n", - " warnings.warn(msg, category=bfe.PreviewWarning)\n" - ] - } - ], + "outputs": [], "source": [ "PROJECT = \"bigframes-dev\" # replace with your project, project needs to be allowlisted go/bq-multimodal-allowlist (internal)\n", "# User must have https://cloud.google.com/bigquery/docs/use-bigquery-dataframes#permissions to use bigframes, BQ connection admin/user to create/use connections, BQ ObjRef permissions for ObjectRef and BQ routines permissions for using transform functions.\n", @@ -101,9 +91,9 @@ "import bigframes\n", "# Setup project\n", "bigframes.options.bigquery.project = PROJECT\n", - "# Flag to enable the feature\n", - "bigframes.options.experiments.blob = True\n", "\n", + "# Display options\n", + "bigframes.options.display.blob_display_width = 300\n", "bigframes.options.display.progress_bar = None\n", "\n", "import bigframes.pandas as bpd" @@ -121,7 +111,7 @@ }, { "cell_type": "code", - "execution_count": 2, + "execution_count": 3, "metadata": { "colab": { "base_uri": "https://localhost:8080/" @@ -154,7 +144,7 @@ }, { "cell_type": "code", - "execution_count": 3, + "execution_count": 4, "metadata": { "colab": { "base_uri": "https://localhost:8080/", @@ -191,23 +181,23 @@ " \n", " \n", " 0\n", - " \n", + " \n", " \n", " \n", " 1\n", - " \n", + " \n", " \n", " \n", " 2\n", - " \n", + " \n", " \n", " \n", " 3\n", - " \n", + " \n", " \n", " \n", " 4\n", - " \n", + " \n", " \n", " \n", "\n", @@ -225,7 +215,7 @@ "[5 rows x 1 columns]" ] }, - "execution_count": 3, + "execution_count": 4, "metadata": {}, "output_type": "execute_result" } @@ -256,7 +246,7 @@ }, { "cell_type": "code", - "execution_count": 4, + "execution_count": 5, "metadata": { "id": "YYYVn7NDH0Me" }, @@ -292,43 +282,43 @@ " \n", " \n", " 0\n", - " \n", + " \n", " alice\n", " image/png\n", - " 1489405\n", - " 2025-03-20 17:44:58+00:00\n", + " 1591240\n", + " 2025-03-20 17:45:04+00:00\n", " \n", " \n", " 1\n", - " \n", + " \n", " bob\n", " image/png\n", - " 1538007\n", - " 2025-03-20 17:44:56+00:00\n", + " 1182951\n", + " 2025-03-20 17:45:02+00:00\n", " \n", " \n", " 2\n", - " \n", + " \n", " bob\n", " image/png\n", - " 1237551\n", - " 2025-03-20 17:45:14+00:00\n", + " 1520884\n", + " 2025-03-20 17:44:55+00:00\n", " \n", " \n", " 3\n", - " \n", + " \n", " alice\n", " image/png\n", - " 1040455\n", - " 2025-03-20 17:44:45+00:00\n", + " 1235401\n", + " 2025-03-20 17:45:19+00:00\n", " \n", " \n", " 4\n", - " \n", + " \n", " bob\n", " image/png\n", - " 1517938\n", - " 2025-03-20 17:45:05+00:00\n", + " 1591923\n", + " 2025-03-20 17:44:47+00:00\n", " \n", " \n", "\n", @@ -344,16 +334,16 @@ "4 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", "\n", " size updated \n", - "0 1489405 2025-03-20 17:44:58+00:00 \n", - "1 1538007 2025-03-20 17:44:56+00:00 \n", - "2 1237551 2025-03-20 17:45:14+00:00 \n", - "3 1040455 2025-03-20 17:44:45+00:00 \n", - "4 1517938 2025-03-20 17:45:05+00:00 \n", + "0 1591240 2025-03-20 17:45:04+00:00 \n", + "1 1182951 2025-03-20 17:45:02+00:00 \n", + "2 1520884 2025-03-20 17:44:55+00:00 \n", + "3 1235401 2025-03-20 17:45:19+00:00 \n", + "4 1591923 2025-03-20 17:44:47+00:00 \n", "\n", "[5 rows x 5 columns]" ] }, - "execution_count": 4, + "execution_count": 5, "metadata": {}, "output_type": "execute_result" } @@ -378,7 +368,7 @@ }, { "cell_type": "code", - "execution_count": 5, + "execution_count": 6, "metadata": { "colab": { "base_uri": "https://localhost:8080/", @@ -391,7 +381,7 @@ { "data": { "text/html": [ - "" + "" ], "text/plain": [ "" @@ -403,7 +393,7 @@ { "data": { "text/html": [ - "" + "" ], "text/plain": [ "" @@ -430,7 +420,7 @@ }, { "cell_type": "code", - "execution_count": 6, + "execution_count": 7, "metadata": { "colab": { "base_uri": "https://localhost:8080/" @@ -469,7 +459,7 @@ }, { "cell_type": "code", - "execution_count": 7, + "execution_count": 8, "metadata": { "colab": { "base_uri": "https://localhost:8080/" @@ -494,7 +484,7 @@ }, { "cell_type": "code", - "execution_count": 8, + "execution_count": 9, "metadata": { "colab": { "base_uri": "https://localhost:8080/", @@ -539,63 +529,63 @@ " \n", " \n", " 0\n", - " \n", + " \n", " alice\n", " image/png\n", - " 1489405\n", - " 2025-03-20 17:44:58+00:00\n", - " \n", - " \n", - " \n", - " \n", + " 1591240\n", + " 2025-03-20 17:45:04+00:00\n", + " \n", + " \n", + " \n", + " \n", " \n", " \n", " 1\n", - " \n", + " \n", " bob\n", " image/png\n", - " 1538007\n", - " 2025-03-20 17:44:56+00:00\n", - " \n", - " \n", - " \n", - " \n", + " 1182951\n", + " 2025-03-20 17:45:02+00:00\n", + " \n", + " \n", + " \n", + " \n", " \n", " \n", " 2\n", - " \n", + " \n", " bob\n", " image/png\n", - " 1237551\n", - " 2025-03-20 17:45:14+00:00\n", - " \n", - " \n", - " \n", - " \n", + " 1520884\n", + " 2025-03-20 17:44:55+00:00\n", + " \n", + " \n", + " \n", + " \n", " \n", " \n", " 3\n", - " \n", + " \n", " alice\n", " image/png\n", - " 1040455\n", - " 2025-03-20 17:44:45+00:00\n", - " \n", - " \n", - " \n", - " \n", + " 1235401\n", + " 2025-03-20 17:45:19+00:00\n", + " \n", + " \n", + " \n", + " \n", " \n", " \n", " 4\n", - " \n", + " \n", " bob\n", " image/png\n", - " 1517938\n", - " 2025-03-20 17:45:05+00:00\n", - " \n", - " \n", - " \n", - " \n", + " 1591923\n", + " 2025-03-20 17:44:47+00:00\n", + " \n", + " \n", + " \n", + " \n", " \n", " \n", "\n", @@ -611,11 +601,11 @@ "4 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", "\n", " size updated \\\n", - "0 1489405 2025-03-20 17:44:58+00:00 \n", - "1 1538007 2025-03-20 17:44:56+00:00 \n", - "2 1237551 2025-03-20 17:45:14+00:00 \n", - "3 1040455 2025-03-20 17:44:45+00:00 \n", - "4 1517938 2025-03-20 17:45:05+00:00 \n", + "0 1591240 2025-03-20 17:45:04+00:00 \n", + "1 1182951 2025-03-20 17:45:02+00:00 \n", + "2 1520884 2025-03-20 17:44:55+00:00 \n", + "3 1235401 2025-03-20 17:45:19+00:00 \n", + "4 1591923 2025-03-20 17:44:47+00:00 \n", "\n", " blurred \\\n", "0 {'uri': 'gs://bigframes_blob_test/image_blur_t... \n", @@ -648,7 +638,7 @@ "[5 rows x 9 columns]" ] }, - "execution_count": 8, + "execution_count": 9, "metadata": {}, "output_type": "execute_result" } @@ -668,7 +658,7 @@ }, { "cell_type": "code", - "execution_count": 9, + "execution_count": 10, "metadata": { "id": "mRUGfcaFVW-3" }, @@ -680,7 +670,7 @@ }, { "cell_type": "code", - "execution_count": 10, + "execution_count": 11, "metadata": { "colab": { "base_uri": "https://localhost:8080/", @@ -694,7 +684,7 @@ "name": "stderr", "output_type": "stream", "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:107: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", + "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:109: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", "`db_dtypes` is a preview feature and subject to change.\n", " warnings.warn(msg, bfe.PreviewWarning)\n" ] @@ -727,13 +717,13 @@ " \n", " \n", " 0\n", - " That's a bag of **rabbit food** from the brand **Fluffy Buns**. The specific product is labeled as \"Ranbhow's trood.flee!\" (which appears to be a playful brand name).\n", - " \n", + " That's a tin of **K9Guard Dog Paw Balm**.\n", + " \n", " \n", " \n", " 1\n", - " That's hay. More specifically, it looks like a type of grass hay, often used as feed for small animals like rabbits, guinea pigs, and chinchillas.\n", - " \n", + " That's a bottle of **K9 Guard Dog Hot Spot Spray**. It's a pet product designed to soothe and protect dogs' hot spots (irritated areas of skin).\n", + " \n", " \n", " \n", "\n", @@ -742,8 +732,9 @@ ], "text/plain": [ " ml_generate_text_llm_result \\\n", - "0 That's a bag of **rabbit food** from the brand... \n", - "1 That's hay. More specifically, it looks like ... \n", + "0 That's a tin of **K9Guard Dog Paw Balm**.\n", + " \n", + "1 That's a bottle of **K9 Guard Dog Hot Spot Spr... \n", "\n", " image \n", "0 {'uri': 'gs://cloud-samples-data/bigquery/tuto... \n", @@ -752,7 +743,7 @@ "[2 rows x 2 columns]" ] }, - "execution_count": 10, + "execution_count": 11, "metadata": {}, "output_type": "execute_result" } @@ -766,7 +757,7 @@ }, { "cell_type": "code", - "execution_count": 11, + "execution_count": 12, "metadata": { "id": "IG3J3HsKhyBY" }, @@ -778,7 +769,7 @@ }, { "cell_type": "code", - "execution_count": 12, + "execution_count": 13, "metadata": { "colab": { "base_uri": "https://localhost:8080/", @@ -792,7 +783,7 @@ "name": "stderr", "output_type": "stream", "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:107: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", + "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:109: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", "`db_dtypes` is a preview feature and subject to change.\n", " warnings.warn(msg, bfe.PreviewWarning)\n" ] @@ -825,13 +816,13 @@ " \n", " \n", " 0\n", - " That's a bag of **Fluffy Buns Rabbit Food**. It's a blend of various ingredients designed as food for rabbits.\n", - " \n", + " That's a tin of **K9Guard Dog Paw Balm**. It's a balm designed to protect and care for a dog's paws.\n", + " \n", " \n", " \n", " 1\n", - " The picture is primarily light green. There are some slightly darker green and yellowish-green shades mixed in, but the overall color is light green.\n", - " \n", + " The background of the picture is light gray. The bottle is predominantly white with teal/light blue accents on the spray nozzle and text. There is a small image of an aloe vera plant on the label.\n", + " \n", " \n", " \n", "\n", @@ -840,8 +831,8 @@ ], "text/plain": [ " ml_generate_text_llm_result \\\n", - "0 That's a bag of **Fluffy Buns Rabbit Food**. ... \n", - "1 The picture is primarily light green. There a... \n", + "0 That's a tin of **K9Guard Dog Paw Balm**. It'... \n", + "1 The background of the picture is light gray. ... \n", "\n", " image \n", "0 {'uri': 'gs://cloud-samples-data/bigquery/tuto... \n", @@ -850,7 +841,7 @@ "[2 rows x 2 columns]" ] }, - "execution_count": 12, + "execution_count": 13, "metadata": {}, "output_type": "execute_result" } @@ -862,7 +853,7 @@ }, { "cell_type": "code", - "execution_count": 13, + "execution_count": 14, "metadata": { "colab": { "base_uri": "https://localhost:8080/", @@ -876,7 +867,11 @@ "name": "stderr", "output_type": "stream", "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:107: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", + "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/log_adapter.py:164: FutureWarning: Since upgrading the default model can cause unintended breakages, the\n", + "default model will be removed in BigFrames 3.0. Please supply an\n", + "explicit model to avoid this message.\n", + " return method(self, *args, **kwargs)\n", + "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:109: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", "`db_dtypes` is a preview feature and subject to change.\n", " warnings.warn(msg, bfe.PreviewWarning)\n" ] @@ -912,21 +907,21 @@ " \n", " \n", " 0\n", - " [ 0.01182145 0.01575819 0.06243018 ... 0.00010706 -0.03063935\n", - " -0.05756916]\n", + " [ 0.00638822 0.01666385 0.00451817 ... -0.02684802 -0.00475593\n", + " -0.01989058]\n", " \n", " \n", " \n", - " {\"access_urls\":{\"expiry_time\":\"2025-04-09T02:36:17Z\",\"read_url\":\"https://storage.googleapis.com/cloud-samples-data/bigquery%2Ftutorials%2Fcymbal-pets%2Fimages%2Ffluffy-buns-rabbit-food.png?X-Goog-Algorithm=GOOG4-RSA-SHA256&X-Goog-Credential=bqcx-1084210331973-pcbl%40gcp-sa-bigquery-condel.iam.gserviceaccount.com%2F20250408%2Fauto%2Fstorage%2Fgoog4_request&X-Goog-Date=20250408T203617Z&X-Goog-Expires=21600&X-Goog-SignedHeaders=host&generation=1742492698095241&X-Goog-Signature=860fb5fbf48778f66f0ec2d141b26dd7ce2de7cacc427491f5cc3f420361770e33936c79552a562a3db31c8e02a9ea73e73ac9c7c379dfa0e213eda456c48243acea3227c3e9e786859b19e2b74718d7c3447f09ba371d77e3df65a9e2936c9b9ad5ad7ba359bfaa9fc3a2785d32359a9d50ee64f90f6e7d3a20a5c13f38f932c83b143dd2abdd31f0b35ab60aa21293d2cbf7ea780b13ef02d6b1f9aa56538a498d3da13798a1cbe2535b118caeb35f1e5be36d09c9593796b5ecf8b171d4915735644a94d19d7e78351e475da7b75f72fc8f88b2607ce8d1fb53d7dc2aa16da3b6ed2130fd700cbc797d1a6cc495833945b3bdfaf933b9a4dc70ff3299ab4f\",\"write_url\":\"\"},\"objectref\":{\"authorizer\":\"bigframes-dev.us.bigframes-default-connection\",\"details\":{\"gcs_metadata\":{\"content_type\":\"image/png\",\"md5_hash\":\"4c01d79182ea7580183a2168076e16b8\",\"size\":1489405,\"updated\":1742492698000000}},\"uri\":\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/fluffy-buns-rabbit-food.png\",\"version\":\"1742492698095241\"}}\n", + " {\"access_urls\":{\"expiry_time\":\"2025-05-06T06:14:15Z\",\"read_url\":\"https://storage.googleapis.com/cloud-samples-data/bigquery%2Ftutorials%2Fcymbal-pets%2Fimages%2Fk9-guard-dog-paw-balm.png?X-Goog-Algorithm=GOOG4-RSA-SHA256&X-Goog-Credential=bqcx-1084210331973-pcbl%40gcp-sa-bigquery-condel.iam.gserviceaccount.com%2F20250506%2Fauto%2Fstorage%2Fgoog4_request&X-Goog-Date=20250506T001415Z&X-Goog-Expires=21600&X-Goog-SignedHeaders=host&generation=1742492703986347&X-Goog-Signature=4ff6735f3e2d1b9aede89b881884c8413f7b5fde1c9c65b140de7eed1bdaca3609cd77e5dfb81e83fbc77adf20dbbec687ab0dd3f9fadd7ffd9961b1eecf14464d8257ab71dc509859eda505877a16a6b97a34bdce2a36b93da14ffb9e4de5b6c60b8c49e249efa56cfa144c5855d92fb67ee8a09f753d9889ff69c8bef70f01224bcfdda66341bd8cd7d56cc4fdc2a5618d1cebc654077890afa5329c05bb8accf440802ad3aeaa624f23725f7e9bec9270d4379c30f7dd4c4ca99752103396f87872e9484fe3efcd6514754d85d825d56bed140e49a15985bee0274fc6d4f5eceecc762144d451454896b5fcda4cade198c2b7bf07e5186556a393b3b85264\",\"write_url\":\"\"},\"objectref\":{\"authorizer\":\"bigframes-dev.us.bigframes-default-connection\",\"details\":{\"gcs_metadata\":{\"content_type\":\"image/png\",\"md5_hash\":\"c46ac29292d8ba244101dcada3ea86d5\",\"size\":1591240,\"updated\":1742492704000000}},\"uri\":\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/k9-guard-dog-paw-balm.png\",\"version\":\"1742492703986347\"}}\n", " \n", " \n", " 1\n", - " [ 0.02554693 0.01508185 0.04101892 ... -0.02417112 -0.01356636\n", - " -0.01999673]\n", + " [ 0.00973672 0.02148364 0.00244308 ... 0.00462242 0.0131027\n", + " -0.00038765]\n", " \n", " \n", " \n", - " {\"access_urls\":{\"expiry_time\":\"2025-04-09T02:36:17Z\",\"read_url\":\"https://storage.googleapis.com/cloud-samples-data/bigquery%2Ftutorials%2Fcymbal-pets%2Fimages%2Ffluffy-buns-guinea-pig-hay.png?X-Goog-Algorithm=GOOG4-RSA-SHA256&X-Goog-Credential=bqcx-1084210331973-pcbl%40gcp-sa-bigquery-condel.iam.gserviceaccount.com%2F20250408%2Fauto%2Fstorage%2Fgoog4_request&X-Goog-Date=20250408T203617Z&X-Goog-Expires=21600&X-Goog-SignedHeaders=host&generation=1742492696656039&X-Goog-Signature=192e852a5296d31a048af459afe3dc539e2bbf90c65bc2997219e7822bd0ca2858b8e04475e12d14d63d295b45e51403b4f4585a6b66c8b0dbc3adf19e135a93687aeff7ba675eec2aeddb4a1cb4d2b83bee22c7c2de80287af63158a85ee56fa1daccbf31bf42d57e5724ea24bdd630a8a1930d70a5d38fb0340d846848039f53bf4efbc21da6df9a7d91fec727385018b159e4fc53fce0b57ab0c77583361bc4e10b2a7080aafa288789240e565eb58cb9abf2bd298732fddaad4f32472110b2607f6b3a21d9fbce1fc3ecb23caf967a4e3ff5101ae29fc6c65b888930a1306c8deb3b569997a0a364325b3ac0350ff671f2682d9a8a4a96bfac28eb9f9fd8\",\"write_url\":\"\"},\"objectref\":{\"authorizer\":\"bigframes-dev.us.bigframes-default-connection\",\"details\":{\"gcs_metadata\":{\"content_type\":\"image/png\",\"md5_hash\":\"0888367a63729f5a42f4a041596f635d\",\"size\":1538007,\"updated\":1742492696000000}},\"uri\":\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/fluffy-buns-guinea-pig-hay.png\",\"version\":\"1742492696656039\"}}\n", + " {\"access_urls\":{\"expiry_time\":\"2025-05-06T06:14:15Z\",\"read_url\":\"https://storage.googleapis.com/cloud-samples-data/bigquery%2Ftutorials%2Fcymbal-pets%2Fimages%2Fk9-guard-dog-hot-spot-spray.png?X-Goog-Algorithm=GOOG4-RSA-SHA256&X-Goog-Credential=bqcx-1084210331973-pcbl%40gcp-sa-bigquery-condel.iam.gserviceaccount.com%2F20250506%2Fauto%2Fstorage%2Fgoog4_request&X-Goog-Date=20250506T001415Z&X-Goog-Expires=21600&X-Goog-SignedHeaders=host&generation=1742492702954473&X-Goog-Signature=4adc1cf96fe21f385782ea26d52faf9093995b334f592ab0b3a791dc47402b5626c82b9c31c3ecd553d6ccf16012d42c0f7dd8e07b641c60515e43e33bc64da60082763f1567bc9ff2f37a19d3e33cf2a87a9872512b9400265b42092c8070254d842a26cc28d0332b86d0bc052a71ea4bd85d026604f81235d9ee367852e912ad5b5ea405023f92c269586a8fa417b1b7ffe8026086f1d9aaf893635334715568025d0f1d7a2108b33cde9a9012e0684763a6b1743e8decbf7bc9e9f582c7f72bcc2bf31254253ef3c21cb2d10c4c1580e98729eb0611edaea2690bf4b4884449a44851fb0f788a7692dc94bf4fe3c2e287566b6b92e4cc880870adfc4093c4\",\"write_url\":\"\"},\"objectref\":{\"authorizer\":\"bigframes-dev.us.bigframes-default-connection\",\"details\":{\"gcs_metadata\":{\"content_type\":\"image/png\",\"md5_hash\":\"ae802a64dfaeaf556609429fcbe02542\",\"size\":1182951,\"updated\":1742492702000000}},\"uri\":\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/k9-guard-dog-hot-spot-spray.png\",\"version\":\"1742492702954473\"}}\n", " \n", " \n", "\n", @@ -935,8 +930,8 @@ ], "text/plain": [ " ml_generate_embedding_result \\\n", - "0 [ 0.01182145 0.01575819 0.06243018 ... 0.00... \n", - "1 [ 0.02554693 0.01508185 0.04101892 ... -0.02... \n", + "0 [ 0.00638822 0.01666385 0.00451817 ... -0.02... \n", + "1 [ 0.00973672 0.02148364 0.00244308 ... 0.00... \n", "\n", " ml_generate_embedding_status ml_generate_embedding_start_sec \\\n", "0 \n", @@ -947,13 +942,13 @@ "1 \n", "\n", " content \n", - "0 {\"access_urls\":{\"expiry_time\":\"2025-04-09T02:3... \n", - "1 {\"access_urls\":{\"expiry_time\":\"2025-04-09T02:3... \n", + "0 {\"access_urls\":{\"expiry_time\":\"2025-05-06T06:1... \n", + "1 {\"access_urls\":{\"expiry_time\":\"2025-05-06T06:1... \n", "\n", "[2 rows x 5 columns]" ] }, - "execution_count": 13, + "execution_count": 14, "metadata": {}, "output_type": "execute_result" } @@ -976,7 +971,7 @@ }, { "cell_type": "code", - "execution_count": 14, + "execution_count": 15, "metadata": { "id": "oDDuYtUm5Yiy" }, @@ -987,7 +982,7 @@ }, { "cell_type": "code", - "execution_count": 15, + "execution_count": 16, "metadata": { "colab": { "base_uri": "https://localhost:8080/" @@ -1011,7 +1006,7 @@ }, { "cell_type": "code", - "execution_count": 16, + "execution_count": 17, "metadata": { "id": "kaPvJATN7zlw" }, @@ -1034,7 +1029,7 @@ "Name: chunked, dtype: string" ] }, - "execution_count": 16, + "execution_count": 17, "metadata": {}, "output_type": "execute_result" } diff --git a/samples/snippets/multimodal_test.py b/samples/snippets/multimodal_test.py index dc326b266e..fc3d60e12e 100644 --- a/samples/snippets/multimodal_test.py +++ b/samples/snippets/multimodal_test.py @@ -19,8 +19,6 @@ def test_multimodal_dataframe(gcs_dst_bucket: str) -> None: # [START bigquery_dataframes_multimodal_dataframe_create] import bigframes - # Flag to enable the feature - bigframes.options.experiments.blob = True # Flags to control preview image/video preview size bigframes.options.experiments.blob_display_width = 300 diff --git a/tests/system/conftest.py b/tests/system/conftest.py index 19f2a79b65..458e8e0ad2 100644 --- a/tests/system/conftest.py +++ b/tests/system/conftest.py @@ -1500,8 +1500,6 @@ def images_uris() -> list[str]: def images_mm_df( images_uris, test_session: bigframes.Session, bq_connection: str ) -> bpd.DataFrame: - bigframes.options.experiments.blob = True - blob_series = bpd.Series(images_uris, session=test_session).str.to_blob( connection=bq_connection ) @@ -1526,8 +1524,6 @@ def pdf_gcs_path() -> str: def pdf_mm_df( pdf_gcs_path, test_session: bigframes.Session, bq_connection: str ) -> bpd.DataFrame: - bigframes.options.experiments.blob = True - return test_session.from_glob_path( pdf_gcs_path, name="pdf", connection=bq_connection ) diff --git a/tests/system/large/blob/test_function.py b/tests/system/large/blob/test_function.py index a2c3f2b85f..5913df8add 100644 --- a/tests/system/large/blob/test_function.py +++ b/tests/system/large/blob/test_function.py @@ -57,8 +57,6 @@ def test_blob_image_blur_to_series( images_output_uris: list[str], test_session: bigframes.Session, ): - bigframes.options.experiments.blob = True - series = bpd.Series(images_output_uris, session=test_session).str.to_blob( connection=bq_connection ) @@ -91,8 +89,6 @@ def test_blob_image_blur_to_folder( images_output_folder: str, images_output_uris: list[str], ): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.image_blur( (8, 8), dst=images_output_folder, connection=bq_connection ) @@ -116,8 +112,6 @@ def test_blob_image_blur_to_folder( def test_blob_image_blur_to_bq(images_mm_df: bpd.DataFrame, bq_connection: str): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.image_blur((8, 8), connection=bq_connection) assert isinstance(actual, bpd.Series) @@ -131,8 +125,6 @@ def test_blob_image_resize_to_series( images_output_uris: list[str], test_session: bigframes.Session, ): - bigframes.options.experiments.blob = True - series = bpd.Series(images_output_uris, session=test_session).str.to_blob( connection=bq_connection ) @@ -165,8 +157,6 @@ def test_blob_image_resize_to_folder( images_output_folder: str, images_output_uris: list[str], ): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.image_resize( (200, 300), dst=images_output_folder, connection=bq_connection ) @@ -190,8 +180,6 @@ def test_blob_image_resize_to_folder( def test_blob_image_resize_to_bq(images_mm_df: bpd.DataFrame, bq_connection: str): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.image_resize( (200, 300), connection=bq_connection ) @@ -207,8 +195,6 @@ def test_blob_image_normalize_to_series( images_output_uris: list[str], test_session: bigframes.Session, ): - bigframes.options.experiments.blob = True - series = bpd.Series(images_output_uris, session=test_session).str.to_blob( connection=bq_connection ) @@ -241,8 +227,6 @@ def test_blob_image_normalize_to_folder( images_output_folder: str, images_output_uris: list[str], ): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.image_normalize( alpha=50.0, beta=150.0, @@ -270,8 +254,6 @@ def test_blob_image_normalize_to_folder( def test_blob_image_normalize_to_bq(images_mm_df: bpd.DataFrame, bq_connection: str): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.image_normalize( alpha=50.0, beta=150.0, norm_type="minmax", connection=bq_connection ) @@ -314,8 +296,6 @@ def test_blob_pdf_extract( bq_connection: str, expected: pd.Series, ): - bigframes.options.experiments.blob = True - actual = ( pdf_mm_df["pdf"] .blob.pdf_extract(connection=bq_connection, verbose=verbose) @@ -366,8 +346,6 @@ def test_blob_pdf_extract( def test_blob_pdf_chunk( pdf_mm_df: bpd.DataFrame, verbose: bool, bq_connection: str, expected: pd.Series ): - bigframes.options.experiments.blob = True - actual = ( pdf_mm_df["pdf"] .blob.pdf_chunk( diff --git a/tests/system/small/blob/test_io.py b/tests/system/small/blob/test_io.py index c496e5d631..806dad71dc 100644 --- a/tests/system/small/blob/test_io.py +++ b/tests/system/small/blob/test_io.py @@ -21,8 +21,6 @@ def test_blob_create_from_uri_str( bq_connection: str, test_session: bigframes.Session, images_uris ): - bigframes.options.experiments.blob = True - uri_series = bpd.Series(images_uris, session=test_session) blob_series = uri_series.str.to_blob(connection=bq_connection) @@ -44,8 +42,6 @@ def test_blob_create_from_uri_str( def test_blob_create_from_glob_path( bq_connection: str, test_session: bigframes.Session, images_gcs_path, images_uris ): - bigframes.options.experiments.blob = True - blob_df = test_session.from_glob_path( images_gcs_path, connection=bq_connection, name="blob_col" ) @@ -74,8 +70,6 @@ def test_blob_create_from_glob_path( def test_blob_create_read_gbq_object_table( bq_connection: str, test_session: bigframes.Session, images_gcs_path, images_uris ): - bigframes.options.experiments.blob = True - obj_table = test_session._create_object_table(images_gcs_path, bq_connection) blob_df = test_session.read_gbq_object_table(obj_table, name="blob_col") diff --git a/tests/system/small/blob/test_properties.py b/tests/system/small/blob/test_properties.py index 767dbe37b7..f6a0c87f24 100644 --- a/tests/system/small/blob/test_properties.py +++ b/tests/system/small/blob/test_properties.py @@ -14,14 +14,11 @@ import pandas as pd -import bigframes import bigframes.dtypes as dtypes import bigframes.pandas as bpd def test_blob_uri(images_uris: list[str], images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.uri().to_pandas() expected = pd.Series(images_uris, name="uri") @@ -31,8 +28,6 @@ def test_blob_uri(images_uris: list[str], images_mm_df: bpd.DataFrame): def test_blob_authorizer(images_mm_df: bpd.DataFrame, bq_connection: str): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.authorizer().to_pandas() expected = pd.Series( [bq_connection.casefold(), bq_connection.casefold()], name="authorizer" @@ -44,8 +39,6 @@ def test_blob_authorizer(images_mm_df: bpd.DataFrame, bq_connection: str): def test_blob_version(images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.version().to_pandas() expected = pd.Series(["1739574332294150", "1739574332271343"], name="version") @@ -55,33 +48,30 @@ def test_blob_version(images_mm_df: bpd.DataFrame): def test_blob_metadata(images_mm_df: bpd.DataFrame): - with bigframes.option_context("experiments.blob", True): - actual = images_mm_df["blob_col"].blob.metadata().to_pandas() - expected = pd.Series( - [ - ( - '{"content_type":"image/jpeg",' - '"md5_hash":"e130ad042261a1883cd2cc06831cf748",' - '"size":338390,' - '"updated":1739574332000000}' - ), - ( - '{"content_type":"image/jpeg",' - '"md5_hash":"e2ae3191ff2b809fd0935f01a537c650",' - '"size":43333,' - '"updated":1739574332000000}' - ), - ], - name="metadata", - dtype=dtypes.JSON_DTYPE, - ) - expected.index = expected.index.astype(dtypes.INT_DTYPE) - pd.testing.assert_series_equal(actual, expected) + actual = images_mm_df["blob_col"].blob.metadata().to_pandas() + expected = pd.Series( + [ + ( + '{"content_type":"image/jpeg",' + '"md5_hash":"e130ad042261a1883cd2cc06831cf748",' + '"size":338390,' + '"updated":1739574332000000}' + ), + ( + '{"content_type":"image/jpeg",' + '"md5_hash":"e2ae3191ff2b809fd0935f01a537c650",' + '"size":43333,' + '"updated":1739574332000000}' + ), + ], + name="metadata", + dtype=dtypes.JSON_DTYPE, + ) + expected.index = expected.index.astype(dtypes.INT_DTYPE) + pd.testing.assert_series_equal(actual, expected) def test_blob_content_type(images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.content_type().to_pandas() expected = pd.Series(["image/jpeg", "image/jpeg"], name="content_type") @@ -91,8 +81,6 @@ def test_blob_content_type(images_mm_df: bpd.DataFrame): def test_blob_md5_hash(images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.md5_hash().to_pandas() expected = pd.Series( ["e130ad042261a1883cd2cc06831cf748", "e2ae3191ff2b809fd0935f01a537c650"], @@ -105,8 +93,6 @@ def test_blob_md5_hash(images_mm_df: bpd.DataFrame): def test_blob_size(images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.size().to_pandas() expected = pd.Series([338390, 43333], name="size") @@ -116,8 +102,6 @@ def test_blob_size(images_mm_df: bpd.DataFrame): def test_blob_updated(images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - actual = images_mm_df["blob_col"].blob.updated().to_pandas() expected = pd.Series( [ diff --git a/tests/system/small/blob/test_urls.py b/tests/system/small/blob/test_urls.py index da972348f2..02a76587f5 100644 --- a/tests/system/small/blob/test_urls.py +++ b/tests/system/small/blob/test_urls.py @@ -12,21 +12,16 @@ # See the License for the specific language governing permissions and # limitations under the License. -import bigframes import bigframes.pandas as bpd def test_blob_read_url(images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - urls = images_mm_df["blob_col"].blob.read_url() assert urls.str.startswith("https://storage.googleapis.com/").all() def test_blob_write_url(images_mm_df: bpd.DataFrame): - bigframes.options.experiments.blob = True - urls = images_mm_df["blob_col"].blob.write_url() assert urls.str.startswith("https://storage.googleapis.com/").all() diff --git a/tests/system/small/ml/test_llm.py b/tests/system/small/ml/test_llm.py index a74642aea3..84698f7821 100644 --- a/tests/system/small/ml/test_llm.py +++ b/tests/system/small/ml/test_llm.py @@ -19,7 +19,6 @@ import pyarrow as pa import pytest -import bigframes from bigframes import exceptions from bigframes.ml import core, llm import bigframes.pandas as bpd @@ -92,8 +91,6 @@ def test_text_embedding_generator_multi_cols_predict_success( def test_create_load_multimodal_embedding_generator_model( dataset_id, session, bq_connection ): - bigframes.options.experiments.blob = True - mm_embedding_model = llm.MultimodalEmbeddingGenerator( connection_name=bq_connection, session=session ) @@ -840,6 +837,5 @@ def test_gemini_preview_model_warnings(model_name): ) def test_text_embedding_generator_no_default_model_warning(model_class): message = "Since upgrading the default model can cause unintended breakages, the\ndefault model will be removed in BigFrames 3.0. Please supply an\nexplicit model to avoid this message." - bigframes.options.experiments.blob = True with pytest.warns(FutureWarning, match=message): model_class(model_name=None) diff --git a/tests/system/small/ml/test_multimodal_llm.py b/tests/system/small/ml/test_multimodal_llm.py index efeadc76cf..5e70a98e07 100644 --- a/tests/system/small/ml/test_multimodal_llm.py +++ b/tests/system/small/ml/test_multimodal_llm.py @@ -16,7 +16,6 @@ import pyarrow as pa import pytest -import bigframes from bigframes.ml import llm import bigframes.pandas as bpd from tests.system import utils @@ -26,8 +25,6 @@ def test_multimodal_embedding_generator_predict_default_params_success( images_mm_df, test_session, bq_connection ): - bigframes.options.experiments.blob = True - text_embedding_model = llm.MultimodalEmbeddingGenerator( connection_name=bq_connection, session=test_session ) @@ -56,8 +53,6 @@ def test_multimodal_embedding_generator_predict_default_params_success( def test_gemini_text_generator_multimodal_input( images_mm_df: bpd.DataFrame, model_name, test_session, bq_connection ): - bigframes.options.experiments.blob = True - gemini_text_generator_model = llm.GeminiTextGenerator( model_name=model_name, connection_name=bq_connection, session=test_session ) @@ -87,8 +82,6 @@ def test_gemini_text_generator_multimodal_input( def test_gemini_text_generator_multimodal_structured_output( images_mm_df: bpd.DataFrame, model_name, test_session, bq_connection ): - bigframes.options.experiments.blob = True - gemini_text_generator_model = llm.GeminiTextGenerator( model_name=model_name, connection_name=bq_connection, session=test_session ) diff --git a/tests/unit/_config/test_experiment_options.py b/tests/unit/_config/test_experiment_options.py index 1e5a8326f7..deeee2e46a 100644 --- a/tests/unit/_config/test_experiment_options.py +++ b/tests/unit/_config/test_experiment_options.py @@ -46,18 +46,3 @@ def test_ai_operators_set_true_shows_warning(): options.ai_operators = True assert options.ai_operators is True - - -def test_blob_default_false(): - options = experiment_options.ExperimentOptions() - - assert options.blob is False - - -def test_blob_set_true_shows_warning(): - options = experiment_options.ExperimentOptions() - - with pytest.warns(bfe.PreviewWarning): - options.blob = True - - assert options.blob is True diff --git a/third_party/bigframes_vendored/pandas/core/config_init.py b/third_party/bigframes_vendored/pandas/core/config_init.py index 4bca3f3c75..51d056a2c8 100644 --- a/third_party/bigframes_vendored/pandas/core/config_init.py +++ b/third_party/bigframes_vendored/pandas/core/config_init.py @@ -84,6 +84,12 @@ memory_usage (bool): This specifies if the memory usage of a DataFrame should be displayed when df.info() is called. Valid values True,False, + blob_display (bool): + Whether to display the blob content in notebook DataFrame preview. Default True. + blob_display_width (int or None): + Width in pixels that the blob constrained to. + blob_display_height (int or None): + Height in pixels that the blob constrained to. """ sampling_options_doc = """ From 9b777a019aa31a115a22289f21c7cd9df07aa8b9 Mon Sep 17 00:00:00 2001 From: Shenyang Cai Date: Tue, 6 May 2025 19:16:20 -0700 Subject: [PATCH 07/36] fix: fix dayofyear doc test (#1701) --- third_party/bigframes_vendored/pandas/core/indexes/accessor.py | 1 - 1 file changed, 1 deletion(-) diff --git a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py index a3c0d59e46..a5004c93d0 100644 --- a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py +++ b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py @@ -87,7 +87,6 @@ def dayofyear(self): 2017-01-02 00:00:00 2 2017-01-03 00:00:00 3 dtype: Int64 - dtype: Int64 Returns: Series: Containing integers indicating the day number. From b16740ef4ad7b1fbf731595238cf087c93c93066 Mon Sep 17 00:00:00 2001 From: rey-esp Date: Wed, 7 May 2025 16:08:07 -0500 Subject: [PATCH 08/36] feat: support forecast_limit_lower_bound and forecast_limit_upper_bound in ARIMA_PLUS (and ARIMA_PLUS_XREG) models (#1305) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * feat: support forecast_limit_lower_bound and forecast_limit_upper_bound in ARIMA_PLUS (and ARIMA_PLUS_XREG) models * update doc string * Update test_forecasting.py - remove upper bound * add TODO * Apply suggestions from code review --------- Co-authored-by: Tim Sweña (Swast) --- .gitignore | 1 - bigframes/ml/forecasting.py | 21 +++++++++++++++++++++ tests/system/large/ml/test_forecasting.py | 3 +++ 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index f7c77e4d3e..d083ea1ddc 100644 --- a/.gitignore +++ b/.gitignore @@ -60,6 +60,5 @@ coverage.xml system_tests/local_test_setup # Make sure a generated file isn't accidentally committed. -demo.ipynb pylintrc pylintrc.test diff --git a/bigframes/ml/forecasting.py b/bigframes/ml/forecasting.py index 7aa8ba5a5f..2e93e5485f 100644 --- a/bigframes/ml/forecasting.py +++ b/bigframes/ml/forecasting.py @@ -36,6 +36,8 @@ "holiday_region": "holidayRegion", "clean_spikes_and_dips": "cleanSpikesAndDips", "adjust_step_changes": "adjustStepChanges", + "forecast_limit_upper_bound": "forecastLimitUpperBound", + "forecast_limit_lower_bound": "forecastLimitLowerBound", "time_series_length_fraction": "timeSeriesLengthFraction", "min_time_series_length": "minTimeSeriesLength", "max_time_series_length": "maxTimeSeriesLength", @@ -78,6 +80,17 @@ class ARIMAPlus(base.SupervisedTrainableWithIdColPredictor): adjust_step_changes (bool, default True): Determines whether or not to perform automatic step change detection and adjustment in the model training pipeline. + forecast_limit_upper_bound (float or None, default None): + The upper bound of the forecasting values. When you specify the ``forecast_limit_upper_bound`` option, all of the forecast values must be less than the specified value. + For example, if you set ``forecast_limit_upper_bound`` to 100, then all of the forecast values are less than 100. + Also, all values greater than or equal to the ``forecast_limit_upper_bound`` value are excluded from modelling. + The forecasting limit ensures that forecasts stay within limits. + + forecast_limit_lower_bound (float or None, default None): + The lower bound of the forecasting values where the minimum value allowed is 0. When you specify the ``forecast_limit_lower_bound`` option, all of the forecast values must be greater than the specified value. + For example, if you set ``forecast_limit_lower_bound`` to 0, then all of the forecast values are larger than 0. Also, all values less than or equal to the ``forecast_limit_lower_bound`` value are excluded from modelling. + The forecasting limit ensures that forecasts stay within limits. + time_series_length_fraction (float or None, default None): The fraction of the interpolated length of the time series that's used to model the time series trend component. All of the time points of the time series are used to model the non-trend component. @@ -106,6 +119,8 @@ def __init__( holiday_region: Optional[str] = None, clean_spikes_and_dips: bool = True, adjust_step_changes: bool = True, + forecast_limit_lower_bound: Optional[float] = None, + forecast_limit_upper_bound: Optional[float] = None, time_series_length_fraction: Optional[float] = None, min_time_series_length: Optional[int] = None, max_time_series_length: Optional[int] = None, @@ -121,6 +136,8 @@ def __init__( self.holiday_region = holiday_region self.clean_spikes_and_dips = clean_spikes_and_dips self.adjust_step_changes = adjust_step_changes + self.forecast_limit_upper_bound = forecast_limit_upper_bound + self.forecast_limit_lower_bound = forecast_limit_lower_bound self.time_series_length_fraction = time_series_length_fraction self.min_time_series_length = min_time_series_length self.max_time_series_length = max_time_series_length @@ -175,6 +192,10 @@ def _bqml_options(self) -> dict: if self.include_drift: options["include_drift"] = True + if self.forecast_limit_upper_bound is not None: + options["forecast_limit_upper_bound"] = self.forecast_limit_upper_bound + if self.forecast_limit_lower_bound is not None: + options["forecast_limit_lower_bound"] = self.forecast_limit_lower_bound return options diff --git a/tests/system/large/ml/test_forecasting.py b/tests/system/large/ml/test_forecasting.py index 7c070fd200..56b93e5338 100644 --- a/tests/system/large/ml/test_forecasting.py +++ b/tests/system/large/ml/test_forecasting.py @@ -154,6 +154,7 @@ def test_arima_plus_model_fit_params( holiday_region="US", clean_spikes_and_dips=False, adjust_step_changes=False, + forecast_limit_lower_bound=0.0, time_series_length_fraction=0.5, min_time_series_length=10, trend_smoothing_window_size=5, @@ -183,6 +184,8 @@ def test_arima_plus_model_fit_params( assert reloaded_model.holiday_region == "US" assert reloaded_model.clean_spikes_and_dips is False assert reloaded_model.adjust_step_changes is False + # TODO(b/391399223): API must return forecastLimitLowerBound for the following assertion + # assert reloaded_model.forecast_limit_lower_bound == 0.0 assert reloaded_model.time_series_length_fraction == 0.5 assert reloaded_model.min_time_series_length == 10 assert reloaded_model.trend_smoothing_window_size == 5 From 48992e26d460832704401bd2a3eedb800c5061cc Mon Sep 17 00:00:00 2001 From: Shobhit Singh Date: Wed, 7 May 2025 14:42:40 -0700 Subject: [PATCH 09/36] docs: include the clean-up step in the udf code snippet (#1698) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * docs: include the clean-up step in the udf code snippet * 🦉 Updates from OwlBot post-processor See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md --------- Co-authored-by: Owl Bot --- samples/snippets/udf.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/samples/snippets/udf.py b/samples/snippets/udf.py index 495cd33e84..5f7ad8a33f 100644 --- a/samples/snippets/udf.py +++ b/samples/snippets/udf.py @@ -112,10 +112,9 @@ def get_hash(input: str) -> str: df_redacted = df[["species", "island", "sex"]].map(get_hash) df_redacted.peek(10) - # [END bigquery_dataframes_udf] - - # Clean up cloud artifacts + # If the BigQuery routine is no longer needed, we can clean it up + # to free up any cloud quota session = bpd.get_global_session() - session.bqclient.delete_routine( - f"{your_bq_dataset_id}.{your_bq_routine_id}", not_found_ok=True - ) + session.bqclient.delete_routine(f"{your_bq_dataset_id}.{your_bq_routine_id}") + + # [END bigquery_dataframes_udf] From 1a658b2aa43c4a7a7f2007a509b0e1401f925dab Mon Sep 17 00:00:00 2001 From: Shobhit Singh Date: Wed, 7 May 2025 15:30:00 -0700 Subject: [PATCH 10/36] feat: improve error message in `Series.apply` for direct udfs (#1673) * feat: improve error message in `Series.apply` for direct udfs * remove stray file, improve message * specify namespace for udf and remote_function in the error message * fix typo --- bigframes/series.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/bigframes/series.py b/bigframes/series.py index 87f1f1d141..37a3723a0a 100644 --- a/bigframes/series.py +++ b/bigframes/series.py @@ -1712,9 +1712,18 @@ def apply( # as a whole. if by_row: raise ValueError( - "A vectorized non-BigFrames BigQuery function can be " - "provided only with by_row=False. For element-wise operation " - "it must be a BigFrames BigQuery function." + "You have passed a function as-is. If your intention is to " + "apply this function in a vectorized way (i.e. to the " + "entire Series as a whole, and you are sure that it " + "performs only the operations that are implemented for a " + "Series (e.g. a chain of arithmetic/logical operations, " + "such as `def foo(s): return s % 2 == 1`), please also " + "specify `by_row=False`. If your function contains " + "arbitrary code, it can only be applied to every element " + "in the Series individually, in which case you must " + "convert it to a BigFrames BigQuery function using " + "`bigframes.pandas.udf`, " + "or `bigframes.pandas.remote_function` before passing." ) try: From 8f115e760d65778dd1b482fcedcd61334c82606d Mon Sep 17 00:00:00 2001 From: Chelsea Lin Date: Wed, 7 May 2025 15:44:06 -0700 Subject: [PATCH 11/36] refactor: support to compile project and add_op (#1677) --- bigframes/core/compile/sqlglot/compiler.py | 10 ++++ .../core/compile/sqlglot/scalar_compiler.py | 46 ++++++++++++++++++- bigframes/core/compile/sqlglot/sqlglot_ir.py | 15 ++++++ bigframes/dataframe.py | 4 +- .../test_compile_projection/out.sql | 12 +++++ .../sqlglot/test_compile_projection.py | 31 +++++++++++++ 6 files changed, 116 insertions(+), 2 deletions(-) create mode 100644 tests/unit/core/compile/sqlglot/snapshots/test_compile_projection/test_compile_projection/out.sql create mode 100644 tests/unit/core/compile/sqlglot/test_compile_projection.py diff --git a/bigframes/core/compile/sqlglot/compiler.py b/bigframes/core/compile/sqlglot/compiler.py index f6d63531da..91d1fa0d85 100644 --- a/bigframes/core/compile/sqlglot/compiler.py +++ b/bigframes/core/compile/sqlglot/compiler.py @@ -163,6 +163,16 @@ def compile_selection( ) return child.select(selected_cols) + @_compile_node.register + def compile_projection( + self, node: nodes.ProjectionNode, child: ir.SQLGlotIR + ) -> ir.SQLGlotIR: + projected_cols: tuple[tuple[str, sge.Expression], ...] = tuple( + (id.sql, scalar_compiler.compile_scalar_expression(expr)) + for expr, id in node.assignments + ) + return child.project(projected_cols) + def _replace_unsupported_ops(node: nodes.BigFrameNode): node = nodes.bottom_up(node, rewrite.rewrite_slice) diff --git a/bigframes/core/compile/sqlglot/scalar_compiler.py b/bigframes/core/compile/sqlglot/scalar_compiler.py index f9197ef745..48aa4c7b0b 100644 --- a/bigframes/core/compile/sqlglot/scalar_compiler.py +++ b/bigframes/core/compile/sqlglot/scalar_compiler.py @@ -18,6 +18,8 @@ import sqlglot.expressions as sge from bigframes.core import expression +import bigframes.core.compile.sqlglot.sqlglot_ir as ir +import bigframes.operations as ops @functools.singledispatch @@ -29,5 +31,47 @@ def compile_scalar_expression( @compile_scalar_expression.register -def compile_deref_op(expr: expression.DerefOp): +def compile_deref_expression(expr: expression.DerefOp) -> sge.Expression: return sge.ColumnDef(this=sge.to_identifier(expr.id.sql, quoted=True)) + + +@compile_scalar_expression.register +def compile_constant_expression( + expr: expression.ScalarConstantExpression, +) -> sge.Expression: + return ir._literal(expr.value, expr.dtype) + + +@compile_scalar_expression.register +def compile_op_expression(expr: expression.OpExpression): + # Non-recursively compiles the children scalar expressions. + args = tuple(map(compile_scalar_expression, expr.inputs)) + + op = expr.op + op_name = expr.op.__class__.__name__ + method_name = f"compile_{op_name.lower()}" + method = globals().get(method_name, None) + if method is None: + raise ValueError( + f"Compilation method '{method_name}' not found for operator '{op_name}'." + ) + + if isinstance(op, ops.UnaryOp): + return method(op, args[0]) + elif isinstance(op, ops.BinaryOp): + return method(op, args[0], args[1]) + elif isinstance(op, ops.TernaryOp): + return method(op, args[0], args[1], args[2]) + elif isinstance(op, ops.NaryOp): + return method(op, *args) + else: + raise TypeError( + f"Operator '{op_name}' has an unrecognized arity or type " + "and cannot be compiled." + ) + + +# TODO: add parenthesize for operators +def compile_addop(op: ops.AddOp, left: sge.Expression, right: sge.Expression): + # TODO: support addop for string dtype. + return sge.Add(this=left, expression=right) diff --git a/bigframes/core/compile/sqlglot/sqlglot_ir.py b/bigframes/core/compile/sqlglot/sqlglot_ir.py index 660576670d..b23349bcbc 100644 --- a/bigframes/core/compile/sqlglot/sqlglot_ir.py +++ b/bigframes/core/compile/sqlglot/sqlglot_ir.py @@ -118,6 +118,21 @@ def select( new_expr = self._encapsulate_as_cte().select(*cols_expr, append=False) return SQLGlotIR(expr=new_expr) + def project( + self, + projected_cols: tuple[tuple[str, sge.Expression], ...], + ) -> SQLGlotIR: + projected_cols_expr = [ + sge.Alias( + this=expr, + alias=sge.to_identifier(id, quoted=self.quoted), + ) + for id, expr in projected_cols + ] + # TODO: some columns are not able to be projected into the same select. + select_expr = self.expr.select(*projected_cols_expr, append=True) + return SQLGlotIR(expr=select_expr) + def _encapsulate_as_cte( self, ) -> sge.Select: diff --git a/bigframes/dataframe.py b/bigframes/dataframe.py index e64aec307a..8e9794a5e3 100644 --- a/bigframes/dataframe.py +++ b/bigframes/dataframe.py @@ -609,7 +609,9 @@ def __getitem__( def _getitem_label(self, key: blocks.Label): col_ids = self._block.cols_matching_label(key) if len(col_ids) == 0: - raise KeyError(key) + raise KeyError( + f"{key} not found in DataFrame columns: {self._block.column_labels}" + ) block = self._block.select_columns(col_ids) if isinstance(self.columns, pandas.MultiIndex): # Multiindex should drop-level if not selecting entire diff --git a/tests/unit/core/compile/sqlglot/snapshots/test_compile_projection/test_compile_projection/out.sql b/tests/unit/core/compile/sqlglot/snapshots/test_compile_projection/test_compile_projection/out.sql new file mode 100644 index 0000000000..3430b6684f --- /dev/null +++ b/tests/unit/core/compile/sqlglot/snapshots/test_compile_projection/test_compile_projection/out.sql @@ -0,0 +1,12 @@ +WITH `bfcte_0` AS ( + SELECT + *, + `bfcol_0` AS `bfcol_3`, + `bfcol_1` + 1 AS `bfcol_4` + FROM UNNEST(ARRAY>[STRUCT(0, 123456789, 0), STRUCT(1, -987654321, 1), STRUCT(2, 314159, 2), STRUCT(3, CAST(NULL AS INT64), 3), STRUCT(4, -234892, 4), STRUCT(5, 55555, 5), STRUCT(6, 101202303, 6), STRUCT(7, -214748367, 7), STRUCT(8, 2, 8)]) +) +SELECT + `bfcol_3` AS `bfcol_5`, + `bfcol_4` AS `bfcol_6`, + `bfcol_2` AS `bfcol_7` +FROM `bfcte_0` \ No newline at end of file diff --git a/tests/unit/core/compile/sqlglot/test_compile_projection.py b/tests/unit/core/compile/sqlglot/test_compile_projection.py new file mode 100644 index 0000000000..be74255649 --- /dev/null +++ b/tests/unit/core/compile/sqlglot/test_compile_projection.py @@ -0,0 +1,31 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +import pandas as pd +import pytest + +import bigframes +import bigframes.pandas as bpd + +pytest.importorskip("pytest_snapshot") + + +def test_compile_projection( + scalars_types_pandas_df: pd.DataFrame, compiler_session: bigframes.Session, snapshot +): + bf_df = bpd.DataFrame( + scalars_types_pandas_df[["int64_col"]], session=compiler_session + ) + bf_df["int64_col"] = bf_df["int64_col"] + 1 + snapshot.assert_match(bf_df.sql, "out.sql") From edaac89c03db1ffc93b56275c765d8a964f7d02d Mon Sep 17 00:00:00 2001 From: Shenyang Cai Date: Wed, 7 May 2025 16:42:46 -0700 Subject: [PATCH 12/36] feat: support () operator between timedeltas (#1702) --- bigframes/operations/numeric_ops.py | 24 +++++++++++++++++-- .../small/operations/test_timedeltas.py | 3 +++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/bigframes/operations/numeric_ops.py b/bigframes/operations/numeric_ops.py index d06d6eb336..9d6749a169 100644 --- a/bigframes/operations/numeric_ops.py +++ b/bigframes/operations/numeric_ops.py @@ -260,9 +260,29 @@ def output_type(self, *input_types: dtypes.ExpressionType) -> dtypes.ExpressionT floordiv_op = FloorDivOp() -pow_op = base_ops.create_binary_op(name="pow", type_signature=op_typing.BINARY_NUMERIC) -mod_op = base_ops.create_binary_op(name="mod", type_signature=op_typing.BINARY_NUMERIC) +@dataclasses.dataclass(frozen=True) +class ModOp(base_ops.BinaryOp): + name: typing.ClassVar[str] = "mod" + + def output_type(self, *input_types: dtypes.ExpressionType) -> dtypes.ExpressionType: + left_type = input_types[0] + right_type = input_types[1] + + if left_type == dtypes.TIMEDELTA_DTYPE and right_type == dtypes.TIMEDELTA_DTYPE: + return dtypes.TIMEDELTA_DTYPE + + if (left_type is None or dtypes.is_numeric(left_type)) and ( + right_type is None or dtypes.is_numeric(right_type) + ): + return dtypes.coerce_to_common(left_type, right_type) + + raise TypeError(f"Cannot mod dtypes {left_type} and {right_type}") + + +mod_op = ModOp() + +pow_op = base_ops.create_binary_op(name="pow", type_signature=op_typing.BINARY_NUMERIC) arctan2_op = base_ops.create_binary_op( name="arctan2", type_signature=op_typing.BINARY_REAL_NUMERIC diff --git a/tests/system/small/operations/test_timedeltas.py b/tests/system/small/operations/test_timedeltas.py index d6b32a3508..18c88db8eb 100644 --- a/tests/system/small/operations/test_timedeltas.py +++ b/tests/system/small/operations/test_timedeltas.py @@ -98,6 +98,7 @@ def _assert_series_equal(actual: pd.Series, expected: pd.Series): (operator.floordiv, "timedelta_col_1", "float_col"), (operator.mul, "timedelta_col_1", "float_col"), (operator.mul, "float_col", "timedelta_col_1"), + (operator.mod, "timedelta_col_1", "timedelta_col_2"), ], ) def test_timedelta_binary_ops_between_series(temporal_dfs, op, col_1, col_2): @@ -120,6 +121,7 @@ def test_timedelta_binary_ops_between_series(temporal_dfs, op, col_1, col_2): (operator.floordiv, "timedelta_col_1", 3), (operator.mul, "timedelta_col_1", 3), (operator.mul, "float_col", pd.Timedelta(1, "s")), + (operator.mod, "timedelta_col_1", pd.Timedelta(7, "s")), ], ) def test_timedelta_binary_ops_series_and_literal(temporal_dfs, op, col, literal): @@ -142,6 +144,7 @@ def test_timedelta_binary_ops_series_and_literal(temporal_dfs, op, col, literal) (operator.floordiv, "float_col", pd.Timedelta(2, "s")), (operator.mul, "timedelta_col_1", 3), (operator.mul, "float_col", pd.Timedelta(1, "s")), + (operator.mod, "timedelta_col_1", pd.Timedelta(7, "s")), ], ) def test_timedelta_binary_ops_literal_and_series(temporal_dfs, op, col, literal): From e3289b7a64ee1400c6cb78e75cff4759d8da8b7a Mon Sep 17 00:00:00 2001 From: TrevorBergeron Date: Wed, 7 May 2025 17:07:03 -0700 Subject: [PATCH 13/36] fix: Fix issues with chunked arrow data (#1700) --- bigframes/core/local_data.py | 24 ++++++++++++++++++++---- tests/unit/test_local_data.py | 20 ++++++++++++++++++++ 2 files changed, 40 insertions(+), 4 deletions(-) diff --git a/bigframes/core/local_data.py b/bigframes/core/local_data.py index d387e0b818..d23f3538dd 100644 --- a/bigframes/core/local_data.py +++ b/bigframes/core/local_data.py @@ -86,7 +86,7 @@ def from_pyarrow(self, table: pa.Table) -> ManagedArrowTable: columns: list[pa.ChunkedArray] = [] fields: list[schemata.SchemaItem] = [] for name, arr in zip(table.column_names, table.columns): - new_arr, bf_type = _adapt_arrow_array(arr) + new_arr, bf_type = _adapt_chunked_array(arr) columns.append(new_arr) fields.append(schemata.SchemaItem(name, bf_type)) @@ -279,10 +279,26 @@ def _adapt_pandas_series( raise e -def _adapt_arrow_array( - array: Union[pa.ChunkedArray, pa.Array] -) -> tuple[Union[pa.ChunkedArray, pa.Array], bigframes.dtypes.Dtype]: +def _adapt_chunked_array( + chunked_array: pa.ChunkedArray, +) -> tuple[pa.ChunkedArray, bigframes.dtypes.Dtype]: + if len(chunked_array.chunks) == 0: + return _adapt_arrow_array(chunked_array.combine_chunks()) + dtype = None + arrays = [] + for chunk in chunked_array.chunks: + array, arr_dtype = _adapt_arrow_array(chunk) + arrays.append(array) + dtype = dtype or arr_dtype + assert dtype is not None + return pa.chunked_array(arrays), dtype + + +def _adapt_arrow_array(array: pa.Array) -> tuple[pa.Array, bigframes.dtypes.Dtype]: """Normalize the array to managed storage types. Preverse shapes, only transforms values.""" + if array.offset != 0: # Offset arrays don't have all operations implemented + return _adapt_arrow_array(pa.concat_arrays([array])) + if pa.types.is_struct(array.type): assert isinstance(array, pa.StructArray) assert isinstance(array.type, pa.StructType) diff --git a/tests/unit/test_local_data.py b/tests/unit/test_local_data.py index 9cd08787c9..bb7330aba4 100644 --- a/tests/unit/test_local_data.py +++ b/tests/unit/test_local_data.py @@ -44,3 +44,23 @@ def test_local_data_well_formed_round_trip(): local_entry = local_data.ManagedArrowTable.from_pandas(pd_data) result = pd.DataFrame(local_entry.itertuples(), columns=pd_data.columns) pandas.testing.assert_frame_equal(pd_data_normalized, result, check_dtype=False) + + +def test_local_data_well_formed_round_trip_chunked(): + pa_table = pa.Table.from_pandas(pd_data, preserve_index=False) + as_rechunked_pyarrow = pa.Table.from_batches(pa_table.to_batches(max_chunksize=2)) + local_entry = local_data.ManagedArrowTable.from_pyarrow(as_rechunked_pyarrow) + result = pd.DataFrame(local_entry.itertuples(), columns=pd_data.columns) + pandas.testing.assert_frame_equal(pd_data_normalized, result, check_dtype=False) + + +def test_local_data_well_formed_round_trip_sliced(): + pa_table = pa.Table.from_pandas(pd_data, preserve_index=False) + as_rechunked_pyarrow = pa.Table.from_batches(pa_table.slice(2, 4).to_batches()) + local_entry = local_data.ManagedArrowTable.from_pyarrow(as_rechunked_pyarrow) + result = pd.DataFrame(local_entry.itertuples(), columns=pd_data.columns) + pandas.testing.assert_frame_equal( + pd_data_normalized[2:4].reset_index(drop=True), + result.reset_index(drop=True), + check_dtype=False, + ) From a84ee75ddd4d9dae1463e505549d74eb4f819338 Mon Sep 17 00:00:00 2001 From: Chelsea Lin Date: Wed, 7 May 2025 18:06:22 -0700 Subject: [PATCH 14/36] feat: support to_strip parameter for str.strip, str.lstrip and str.rstrip (#1705) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This update introduces `to_strip` parameters for the `str.strip`, `str.lstrip`, and `str.rstrip` methods. Additionally, it resolves an issue where newlines were not stripped when `to_strip` was None, ensuring compatibility with pandas behavior. Fixes internal issue 416077777🦕 --- bigframes/core/compile/scalar_op_compiler.py | 47 +++++++++---- bigframes/operations/__init__.py | 11 ++-- bigframes/operations/string_ops.py | 39 +++++++---- bigframes/operations/strings.py | 18 +++-- tests/system/small/operations/test_strings.py | 66 +++++++++++++++++++ .../pandas/core/strings/accessor.py | 39 +++++++++-- 6 files changed, 177 insertions(+), 43 deletions(-) diff --git a/bigframes/core/compile/scalar_op_compiler.py b/bigframes/core/compile/scalar_op_compiler.py index 2b9208137b..e2dfa38ce1 100644 --- a/bigframes/core/compile/scalar_op_compiler.py +++ b/bigframes/core/compile/scalar_op_compiler.py @@ -456,9 +456,19 @@ def upper_op_impl(x: ibis_types.Value): return typing.cast(ibis_types.StringValue, x).upper() -@scalar_op_compiler.register_unary_op(ops.strip_op) -def strip_op_impl(x: ibis_types.Value): - return typing.cast(ibis_types.StringValue, x).strip() +@scalar_op_compiler.register_unary_op(ops.StrLstripOp, pass_op=True) +def str_lstrip_op_impl(x: ibis_types.Value, op: ops.StrStripOp): + return str_lstrip_op(x, to_strip=op.to_strip) + + +@scalar_op_compiler.register_unary_op(ops.StrRstripOp, pass_op=True) +def str_rstrip_op_impl(x: ibis_types.Value, op: ops.StrRstripOp): + return str_rstrip_op(x, to_strip=op.to_strip) + + +@scalar_op_compiler.register_unary_op(ops.StrStripOp, pass_op=True) +def str_strip_op_impl(x: ibis_types.Value, op: ops.StrStripOp): + return str_strip_op(x, to_strip=op.to_strip) @scalar_op_compiler.register_unary_op(ops.isnumeric_op) @@ -519,16 +529,6 @@ def isupper_op_impl(x: ibis_types.Value): ).re_search(r"\p{Ll}|\p{Lt}") -@scalar_op_compiler.register_unary_op(ops.rstrip_op) -def rstrip_op_impl(x: ibis_types.Value): - return typing.cast(ibis_types.StringValue, x).rstrip() - - -@scalar_op_compiler.register_unary_op(ops.lstrip_op) -def lstrip_op_impl(x: ibis_types.Value): - return typing.cast(ibis_types.StringValue, x).lstrip() - - @scalar_op_compiler.register_unary_op(ops.capitalize_op) def capitalize_op_impl(x: ibis_types.Value): return typing.cast(ibis_types.StringValue, x).capitalize() @@ -2077,3 +2077,24 @@ def obj_make_ref(uri: str, authorizer: str) -> _OBJ_REF_IBIS_DTYPE: # type: ign @ibis_udf.scalar.builtin(name="OBJ.GET_ACCESS_URL") def obj_get_access_url(obj_ref: _OBJ_REF_IBIS_DTYPE, mode: ibis_dtypes.String) -> ibis_dtypes.JSON: # type: ignore """Get access url (as ObjectRefRumtime JSON) from ObjectRef.""" + + +@ibis_udf.scalar.builtin(name="ltrim") +def str_lstrip_op( # type: ignore[empty-body] + x: ibis_dtypes.String, to_strip: ibis_dtypes.String +) -> ibis_dtypes.String: + """Remove leading and trailing characters.""" + + +@ibis_udf.scalar.builtin(name="rtrim") +def str_rstrip_op( # type: ignore[empty-body] + x: ibis_dtypes.String, to_strip: ibis_dtypes.String +) -> ibis_dtypes.String: + """Remove leading and trailing characters.""" + + +@ibis_udf.scalar.builtin(name="trim") +def str_strip_op( # type: ignore[empty-body] + x: ibis_dtypes.String, to_strip: ibis_dtypes.String +) -> ibis_dtypes.String: + """Remove leading and trailing characters.""" diff --git a/bigframes/operations/__init__.py b/bigframes/operations/__init__.py index c8ccaf2a25..0f9b64b760 100644 --- a/bigframes/operations/__init__.py +++ b/bigframes/operations/__init__.py @@ -167,11 +167,9 @@ isupper_op, len_op, lower_op, - lstrip_op, RegexReplaceStrOp, ReplaceStrOp, reverse_op, - rstrip_op, StartsWithOp, strconcat_op, StrContainsOp, @@ -180,10 +178,12 @@ StrFindOp, StrGetOp, StringSplitOp, - strip_op, + StrLstripOp, StrPadOp, StrRepeatOp, + StrRstripOp, StrSliceOp, + StrStripOp, upper_op, ZfillOp, ) @@ -237,11 +237,9 @@ "isupper_op", "len_op", "lower_op", - "lstrip_op", "RegexReplaceStrOp", "ReplaceStrOp", "reverse_op", - "rstrip_op", "StartsWithOp", "strconcat_op", "StrContainsOp", @@ -249,11 +247,14 @@ "StrExtractOp", "StrFindOp", "StrGetOp", + "StrLstripOp", "StringSplitOp", "strip_op", "StrPadOp", "StrRepeatOp", + "StrRstripOp", "StrSliceOp", + "StrStripOp", "upper_op", "ZfillOp", # Date ops diff --git a/bigframes/operations/string_ops.py b/bigframes/operations/string_ops.py index b2ce0706ce..a2755f6654 100644 --- a/bigframes/operations/string_ops.py +++ b/bigframes/operations/string_ops.py @@ -41,10 +41,6 @@ name="upper", type_signature=op_typing.STRING_TRANSFORM ) -strip_op = base_ops.create_unary_op( - name="strip", type_signature=op_typing.STRING_TRANSFORM -) - isalnum_op = base_ops.create_unary_op( name="isalnum", type_signature=op_typing.STRING_PREDICATE ) @@ -77,14 +73,6 @@ name="isupper", type_signature=op_typing.STRING_PREDICATE ) -rstrip_op = base_ops.create_unary_op( - name="rstrip", type_signature=op_typing.STRING_TRANSFORM -) - -lstrip_op = base_ops.create_unary_op( - name="lstrip", type_signature=op_typing.STRING_TRANSFORM -) - capitalize_op = base_ops.create_unary_op( name="capitalize", type_signature=op_typing.STRING_TRANSFORM ) @@ -128,6 +116,33 @@ def output_type(self, *input_types): return op_typing.STRING_TRANSFORM.output_type(input_types[0]) +@dataclasses.dataclass(frozen=True) +class StrStripOp(base_ops.UnaryOp): + name: typing.ClassVar[str] = "str_strip" + to_strip: str + + def output_type(self, *input_types): + return op_typing.STRING_TRANSFORM.output_type(input_types[0]) + + +@dataclasses.dataclass(frozen=True) +class StrLstripOp(base_ops.UnaryOp): + name: typing.ClassVar[str] = "str_lstrip" + to_strip: str + + def output_type(self, *input_types): + return op_typing.STRING_TRANSFORM.output_type(input_types[0]) + + +@dataclasses.dataclass(frozen=True) +class StrRstripOp(base_ops.UnaryOp): + name: typing.ClassVar[str] = "str_rstrip" + to_strip: str + + def output_type(self, *input_types): + return op_typing.STRING_TRANSFORM.output_type(input_types[0]) + + @dataclasses.dataclass(frozen=True) class ReplaceStrOp(base_ops.UnaryOp): name: typing.ClassVar[str] = "str_replace" diff --git a/bigframes/operations/strings.py b/bigframes/operations/strings.py index 529dd87797..a8430b0b0e 100644 --- a/bigframes/operations/strings.py +++ b/bigframes/operations/strings.py @@ -91,8 +91,10 @@ def slice( ) -> series.Series: return self._apply_unary_op(ops.StrSliceOp(start=start, end=stop)) - def strip(self) -> series.Series: - return self._apply_unary_op(ops.strip_op) + def strip(self, to_strip: Optional[str] = None) -> series.Series: + return self._apply_unary_op( + ops.StrStripOp(to_strip=" \n\t" if to_strip is None else to_strip) + ) def upper(self) -> series.Series: return self._apply_unary_op(ops.upper_op) @@ -135,11 +137,15 @@ def isupper( ) -> series.Series: return self._apply_unary_op(ops.isupper_op) - def rstrip(self) -> series.Series: - return self._apply_unary_op(ops.rstrip_op) + def rstrip(self, to_strip: Optional[str] = None) -> series.Series: + return self._apply_unary_op( + ops.StrRstripOp(to_strip=" \n\t" if to_strip is None else to_strip) + ) - def lstrip(self) -> series.Series: - return self._apply_unary_op(ops.lstrip_op) + def lstrip(self, to_strip: Optional[str] = None) -> series.Series: + return self._apply_unary_op( + ops.StrLstripOp(to_strip=" \n\t" if to_strip is None else to_strip) + ) def repeat(self, repeats: int) -> series.Series: return self._apply_unary_op(ops.StrRepeatOp(repeats=repeats)) diff --git a/tests/system/small/operations/test_strings.py b/tests/system/small/operations/test_strings.py index bb328360ee..e4824875b4 100644 --- a/tests/system/small/operations/test_strings.py +++ b/tests/system/small/operations/test_strings.py @@ -265,6 +265,28 @@ def test_strip(scalars_dfs): ) +@pytest.mark.parametrize( + ("to_strip"), + [ + pytest.param(None, id="none"), + pytest.param(" ", id="space"), + pytest.param(" \n", id="space_newline"), + pytest.param("123.!? \n\t", id="multiple_chars"), + ], +) +def test_strip_w_to_strip(to_strip): + s = bpd.Series(["1. Ant. ", "2. Bee!\n", "3. Cat?\t", bpd.NA]) + pd_s = s.to_pandas() + + bf_result = s.str.strip(to_strip=to_strip).to_pandas() + pd_result = pd_s.str.strip(to_strip=to_strip) + + assert_series_equal( + pd_result, + bf_result, + ) + + def test_upper(scalars_dfs): scalars_df, scalars_pandas_df = scalars_dfs col_name = "string_col" @@ -387,6 +409,28 @@ def test_rstrip(scalars_dfs): ) +@pytest.mark.parametrize( + ("to_strip"), + [ + pytest.param(None, id="none"), + pytest.param(" ", id="space"), + pytest.param(" \n", id="space_newline"), + pytest.param("123.!? \n\t", id="multiple_chars"), + ], +) +def test_rstrip_w_to_strip(to_strip): + s = bpd.Series(["1. Ant. ", "2. Bee!\n", "3. Cat?\t", bpd.NA]) + pd_s = s.to_pandas() + + bf_result = s.str.rstrip(to_strip=to_strip).to_pandas() + pd_result = pd_s.str.rstrip(to_strip=to_strip) + + assert_series_equal( + pd_result, + bf_result, + ) + + def test_lstrip(scalars_dfs): scalars_df, scalars_pandas_df = scalars_dfs col_name = "string_col" @@ -400,6 +444,28 @@ def test_lstrip(scalars_dfs): ) +@pytest.mark.parametrize( + ("to_strip"), + [ + pytest.param(None, id="none"), + pytest.param(" ", id="space"), + pytest.param(" \n", id="space_newline"), + pytest.param("123.!? \n\t", id="multiple_chars"), + ], +) +def test_lstrip_w_to_strip(to_strip): + s = bpd.Series(["1. Ant. ", "2. Bee!\n", "3. Cat?\t", bpd.NA]) + pd_s = s.to_pandas() + + bf_result = s.str.lstrip(to_strip=to_strip).to_pandas() + pd_result = pd_s.str.lstrip(to_strip=to_strip) + + assert_series_equal( + pd_result, + bf_result, + ) + + @pytest.mark.parametrize(["repeats"], [(5,), (0,), (1,)]) def test_repeat(scalars_dfs, repeats): scalars_df, scalars_pandas_df = scalars_dfs diff --git a/third_party/bigframes_vendored/pandas/core/strings/accessor.py b/third_party/bigframes_vendored/pandas/core/strings/accessor.py index bd5e78f415..9f3d87ecb7 100644 --- a/third_party/bigframes_vendored/pandas/core/strings/accessor.py +++ b/third_party/bigframes_vendored/pandas/core/strings/accessor.py @@ -239,7 +239,7 @@ def slice(self, start=None, stop=None): raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) - def strip(self): + def strip(self, to_strip: typing.Optional[str] = None): """Remove leading and trailing characters. Strip whitespaces (including newlines) or a set of specified characters @@ -252,22 +252,35 @@ def strip(self): >>> import bigframes.pandas as bpd >>> bpd.options.display.progress_bar = None - >>> s = bpd.Series(['Ant', ' Bee ', '\\tCat\\n', bpd.NA]) + >>> s = bpd.Series(['1. Ant.', ' 2. Bee? ', '\\t3. Cat!\\n', bpd.NA]) >>> s - 0 Ant - 1 Bee - 2 Cat + 0 1. Ant. + 1 2. Bee? + 2 3. Cat! 3 dtype: string >>> s.str.strip() + 0 1. Ant. + 1 2. Bee? + 2 3. Cat! + 3 + dtype: string + + >>> s.str.strip('123.!? \\n\\t') 0 Ant 1 Bee 2 Cat 3 dtype: string + Args: + to_strip (str, default None): + Specifying the set of characters to be removed. All combinations + of this set of characters will be stripped. If None then + whitespaces are removed. + Returns: bigframes.series.Series: Series or Index without leading and trailing characters. @@ -529,7 +542,7 @@ def isdecimal(self): raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) - def rstrip(self): + def rstrip(self, to_strip: typing.Optional[str] = None): """Remove trailing characters. Strip whitespaces (including newlines) or a set of specified characters @@ -558,13 +571,19 @@ def rstrip(self): 3 dtype: string + Args: + to_strip (str, default None): + Specifying the set of characters to be removed. All combinations + of this set of characters will be stripped. If None then + whitespaces are removed. + Returns: bigframes.series.Series: Series without trailing characters. """ raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) - def lstrip(self): + def lstrip(self, to_strip: typing.Optional[str] = None): """Remove leading characters. Strip whitespaces (including newlines) or a set of specified characters @@ -594,6 +613,12 @@ def lstrip(self): 3 dtype: string + Args: + to_strip (str, default None): + Specifying the set of characters to be removed. All combinations + of this set of characters will be stripped. If None then + whitespaces are removed. + Returns: bigframes.series.Series: Series without leading characters. """ From f661a5f8febf2cb5b7a69b7976ae3096eec1795d Mon Sep 17 00:00:00 2001 From: Shuowei Li Date: Wed, 7 May 2025 19:18:08 -0700 Subject: [PATCH 15/36] test: enable gemini-1.5-pro-002 tests (#1707) --- tests/system/load/test_llm.py | 2 +- tests/system/small/ml/test_llm.py | 14 +++++++------- tests/system/small/ml/test_multimodal_llm.py | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/system/load/test_llm.py b/tests/system/load/test_llm.py index e3aead5425..abb199b8ab 100644 --- a/tests/system/load/test_llm.py +++ b/tests/system/load/test_llm.py @@ -41,7 +41,7 @@ def llm_remote_text_df(session, llm_remote_text_pandas_df): @pytest.mark.parametrize( "model_name", ( - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-002", ), ) diff --git a/tests/system/small/ml/test_llm.py b/tests/system/small/ml/test_llm.py index 84698f7821..3f06a02469 100644 --- a/tests/system/small/ml/test_llm.py +++ b/tests/system/small/ml/test_llm.py @@ -111,7 +111,7 @@ def test_create_load_multimodal_embedding_generator_model( "gemini-1.5-pro-preview-0514", "gemini-1.5-flash-preview-0514", "gemini-1.5-pro-001", - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-001", "gemini-1.5-flash-002", "gemini-2.0-flash-exp", @@ -146,7 +146,7 @@ def test_create_load_gemini_text_generator_model( "gemini-1.5-pro-preview-0514", "gemini-1.5-flash-preview-0514", "gemini-1.5-pro-001", - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-001", "gemini-1.5-flash-002", "gemini-2.0-flash-exp", @@ -173,7 +173,7 @@ def test_gemini_text_generator_predict_default_params_success( "gemini-1.5-pro-preview-0514", "gemini-1.5-flash-preview-0514", "gemini-1.5-pro-001", - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-001", "gemini-1.5-flash-002", "gemini-2.0-flash-exp", @@ -202,7 +202,7 @@ def test_gemini_text_generator_predict_with_params_success( "gemini-1.5-pro-preview-0514", "gemini-1.5-flash-preview-0514", "gemini-1.5-pro-001", - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-001", "gemini-1.5-flash-002", "gemini-2.0-flash-exp", @@ -233,7 +233,7 @@ def test_gemini_text_generator_multi_cols_predict_success( "gemini-1.5-pro-preview-0514", "gemini-1.5-flash-preview-0514", "gemini-1.5-pro-001", - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-001", "gemini-1.5-flash-002", "gemini-2.0-flash-exp", @@ -756,7 +756,7 @@ def test_text_embedding_generator_retry_no_progress(session, bq_connection): @pytest.mark.parametrize( "model_name", ( - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-002", "gemini-2.0-flash-001", "gemini-2.0-flash-lite-001", @@ -786,7 +786,7 @@ def test_llm_gemini_score(llm_fine_tune_df_default_index, model_name): @pytest.mark.parametrize( "model_name", ( - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-002", "gemini-2.0-flash-001", "gemini-2.0-flash-lite-001", diff --git a/tests/system/small/ml/test_multimodal_llm.py b/tests/system/small/ml/test_multimodal_llm.py index 5e70a98e07..19ec3d7e14 100644 --- a/tests/system/small/ml/test_multimodal_llm.py +++ b/tests/system/small/ml/test_multimodal_llm.py @@ -42,7 +42,7 @@ def test_multimodal_embedding_generator_predict_default_params_success( "model_name", ( "gemini-1.5-pro-001", - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-001", "gemini-1.5-flash-002", "gemini-2.0-flash-exp", @@ -71,7 +71,7 @@ def test_gemini_text_generator_multimodal_input( "model_name", ( "gemini-1.5-pro-001", - # "gemini-1.5-pro-002", + "gemini-1.5-pro-002", "gemini-1.5-flash-001", "gemini-1.5-flash-002", "gemini-2.0-flash-exp", From 24b37aece60460aabecce306397eb1bf6686f8a7 Mon Sep 17 00:00:00 2001 From: rey-esp Date: Thu, 8 May 2025 10:05:05 -0500 Subject: [PATCH 16/36] docs: add snippets for Matrix Factorization tutorials (#1630) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * docs: add matrix_factorization snippets * incomplete mf snippets * prep implicit * near complete tutorial * implicit create * add doc note * complete explicit tutorial * remove implicit snippets * Update samples/snippets/mf_explicit_model_test.py * add snippets to create dataset and movielens tables * correct the region tags * correct more region tags * Update samples/snippets/mf_explicit_model_test.py Co-authored-by: Tim Sweña (Swast) * Update samples/snippets/mf_explicit_model_test.py Co-authored-by: Tim Sweña (Swast) * update evaluate section --------- Co-authored-by: Tim Sweña (Swast) --- samples/snippets/mf_explicit_model_test.py | 162 +++++++++++++++++++++ 1 file changed, 162 insertions(+) create mode 100644 samples/snippets/mf_explicit_model_test.py diff --git a/samples/snippets/mf_explicit_model_test.py b/samples/snippets/mf_explicit_model_test.py new file mode 100644 index 0000000000..fb54b7271c --- /dev/null +++ b/samples/snippets/mf_explicit_model_test.py @@ -0,0 +1,162 @@ +# Copyright 2024 Google LLC +# +# Licensed under the Apache License, Version 2.0 (t +# you may not use this file except in compliance wi +# 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 +# distributed under the License is distributed on a +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, eit +# See the License for the specific language governi +# limitations under the License. + + +def test_explicit_matrix_factorization(random_model_id: str) -> None: + your_model_id = random_model_id + + # [START bigquery_dataframes_bqml_mf_explicit_create_dataset] + import google.cloud.bigquery + + bqclient = google.cloud.bigquery.Client() + bqclient.create_dataset("bqml_tutorial", exists_ok=True) + # [END bigquery_dataframes_bqml_mf_explicit_create_dataset] + + # [START bigquery_dataframes_bqml_mf_explicit_upload_movielens] + import io + import zipfile + + import google.api_core.exceptions + import requests + + try: + # Check if you've already created the Movielens tables to avoid downloading + # and uploading the dataset unnecessarily. + bqclient.get_table("bqml_tutorial.ratings") + bqclient.get_table("bqml_tutorial.movies") + except google.api_core.exceptions.NotFound: + # Download the https://grouplens.org/datasets/movielens/1m/ dataset. + ml1m = requests.get("http://files.grouplens.org/datasets/movielens/ml-1m.zip") + ml1m_file = io.BytesIO(ml1m.content) + ml1m_zip = zipfile.ZipFile(ml1m_file) + + # Upload the ratings data into the ratings table. + with ml1m_zip.open("ml-1m/ratings.dat") as ratings_file: + ratings_content = ratings_file.read() + + ratings_csv = io.BytesIO(ratings_content.replace(b"::", b",")) + ratings_config = google.cloud.bigquery.LoadJobConfig() + ratings_config.source_format = "CSV" + ratings_config.write_disposition = "WRITE_TRUNCATE" + ratings_config.schema = [ + google.cloud.bigquery.SchemaField("user_id", "INT64"), + google.cloud.bigquery.SchemaField("item_id", "INT64"), + google.cloud.bigquery.SchemaField("rating", "FLOAT64"), + google.cloud.bigquery.SchemaField("timestamp", "TIMESTAMP"), + ] + bqclient.load_table_from_file( + ratings_csv, "bqml_tutorial.ratings", job_config=ratings_config + ).result() + + # Upload the movie data into the movies table. + with ml1m_zip.open("ml-1m/movies.dat") as movies_file: + movies_content = movies_file.read() + + movies_csv = io.BytesIO(movies_content.replace(b"::", b"@")) + movies_config = google.cloud.bigquery.LoadJobConfig() + movies_config.source_format = "CSV" + movies_config.field_delimiter = "@" + movies_config.write_disposition = "WRITE_TRUNCATE" + movies_config.schema = [ + google.cloud.bigquery.SchemaField("movie_id", "INT64"), + google.cloud.bigquery.SchemaField("movie_title", "STRING"), + google.cloud.bigquery.SchemaField("genre", "STRING"), + ] + bqclient.load_table_from_file( + movies_csv, "bqml_tutorial.movies", job_config=movies_config + ).result() + # [END bigquery_dataframes_bqml_mf_explicit_upload_movielens] + + # [START bigquery_dataframes_bqml_mf_explicit_create] + from bigframes.ml import decomposition + import bigframes.pandas as bpd + + # Load data from BigQuery + bq_df = bpd.read_gbq( + "bqml_tutorial.ratings", columns=("user_id", "item_id", "rating") + ) + + # Create the Matrix Factorization model + model = decomposition.MatrixFactorization( + num_factors=34, + feedback_type="explicit", + user_col="user_id", + item_col="item_id", + rating_col="rating", + l2_reg=9.83, + ) + model.fit(bq_df) + model.to_gbq( + your_model_id, replace=True # For example: "bqml_tutorial.mf_explicit" + ) + # [END bigquery_dataframes_bqml_mf_explicit_create] + # [START bigquery_dataframes_bqml_mf_explicit_evaluate] + # Evaluate the model using the score() function + model.score(bq_df) + # Output: + # mean_absolute_error mean_squared_error mean_squared_log_error median_absolute_error r2_score explained_variance + # 0.485403 0.395052 0.025515 0.390573 0.68343 0.68343 + # [END bigquery_dataframes_bqml_mf_explicit_evaluate] + # [START bigquery_dataframes_bqml_mf_explicit_recommend_df] + # Use predict() to get the predicted rating for each movie for 5 users + subset = bq_df[["user_id"]].head(5) + predicted = model.predict(subset) + print(predicted) + # Output: + # predicted_rating user_id item_id rating + # 0 4.206146 4354 968 4.0 + # 1 4.853099 3622 3521 5.0 + # 2 2.679067 5543 920 2.0 + # 3 4.323458 445 3175 5.0 + # 4 3.476911 5535 235 4.0 + # [END bigquery_dataframes_bqml_mf_explicit_recommend_df] + # [START bigquery_dataframes_bqml_mf_explicit_recommend_model] + # import bigframes.bigquery as bbq + + # Load movies + movies = bpd.read_gbq("bqml_tutorial.movies") + + # Merge the movies df with the previously created predicted df + merged_df = bpd.merge(predicted, movies, left_on="item_id", right_on="movie_id") + + # Separate users and predicted data, setting the index to 'movie_id' + users = merged_df[["user_id", "movie_id"]].set_index("movie_id") + + # Take the predicted data and sort it in descending order by 'predicted_rating', setting the index to 'movie_id' + sort_data = ( + merged_df[["movie_title", "genre", "predicted_rating", "movie_id"]] + .sort_values(by="predicted_rating", ascending=False) + .set_index("movie_id") + ) + + # re-merge the separated dfs by index + merged_user = sort_data.join(users, how="outer") + + # group the users and set the user_id as the index + merged_user.groupby("user_id").head(5).set_index("user_id").sort_index() + print(merged_user) + # Output: + # movie_title genre predicted_rating + # user_id + # 1 Saving Private Ryan (1998) Action|Drama|War 5.19326 + # 1 Fargo (1996) Crime|Drama|Thriller 4.996954 + # 1 Driving Miss Daisy (1989) Drama 4.983671 + # 1 Ben-Hur (1959) Action|Adventure|Drama 4.877622 + # 1 Schindler's List (1993) Drama|War 4.802336 + # 2 Saving Private Ryan (1998) Action|Drama|War 5.19326 + # 2 Braveheart (1995) Action|Drama|War 5.174145 + # 2 Gladiator (2000) Action|Drama 5.066372 + # 2 On Golden Pond (1981) Drama 5.01198 + # 2 Driving Miss Daisy (1989) Drama 4.983671 + # [END bigquery_dataframes_bqml_mf_explicit_recommend_model] From 597d8178048b203cea4777f29b1ce95de7b0670e Mon Sep 17 00:00:00 2001 From: Garrett Wu <6505921+GarrettWu@users.noreply.github.com> Date: Thu, 8 May 2025 08:52:29 -0700 Subject: [PATCH 17/36] deps: move bigtable and pubsub to extras (#1696) * deps: move bigtable and pubsub to extras * 2025 * fix mypy --- setup.py | 9 +- tests/system/large/streaming/test_bigtable.py | 104 ++++++++++++++++++ .../test_pubsub.py} | 87 +-------------- 3 files changed, 115 insertions(+), 85 deletions(-) create mode 100644 tests/system/large/streaming/test_bigtable.py rename tests/system/large/{test_streaming.py => streaming/test_pubsub.py} (57%) diff --git a/setup.py b/setup.py index 489d9aacd9..edd8e63e65 100644 --- a/setup.py +++ b/setup.py @@ -39,8 +39,6 @@ "gcsfs >=2023.3.0", "geopandas >=0.12.2", "google-auth >=2.15.0,<3.0", - "google-cloud-bigtable >=2.24.0", - "google-cloud-pubsub >=2.21.4", "google-cloud-bigquery[bqstorage,pandas] >=3.31.0", # 2.30 needed for arrow support. "google-cloud-bigquery-storage >= 2.30.0, < 3.0.0", @@ -72,7 +70,12 @@ ] extras = { # Optional test dependencies packages. If they're missed, may skip some tests. - "tests": ["freezegun", "pytest-snapshot"], + "tests": [ + "freezegun", + "pytest-snapshot", + "google-cloud-bigtable >=2.24.0", + "google-cloud-pubsub >=2.21.4", + ], # used for local engine, which is only needed for unit tests at present. "polars": ["polars >= 1.7.0"], "scikit-learn": ["scikit-learn>=1.2.2"], diff --git a/tests/system/large/streaming/test_bigtable.py b/tests/system/large/streaming/test_bigtable.py new file mode 100644 index 0000000000..e57b7e6e0e --- /dev/null +++ b/tests/system/large/streaming/test_bigtable.py @@ -0,0 +1,104 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +import time +from typing import Generator +import uuid + +import pytest + +import bigframes + +pytest.importorskip("google.cloud.bigtable") + +from google.cloud import bigtable # noqa +from google.cloud.bigtable import column_family, instance, table # noqa + + +@pytest.fixture(scope="session") +def bigtable_instance(session_load: bigframes.Session) -> instance.Instance: + client = bigtable.Client(project=session_load._project, admin=True) + + instance_name = "streaming-testing-instance" + bt_instance = instance.Instance( + instance_name, + client, + ) + + if not bt_instance.exists(): + cluster_id = "streaming-testing-instance-c1" + cluster = bt_instance.cluster( + cluster_id, + location_id="us-west1-a", + serve_nodes=1, + ) + operation = bt_instance.create( + clusters=[cluster], + ) + operation.result(timeout=480) + return bt_instance + + +@pytest.fixture(scope="function") +def bigtable_table( + bigtable_instance: instance.Instance, +) -> Generator[table.Table, None, None]: + table_id = "bigframes_test_" + uuid.uuid4().hex + bt_table = table.Table( + table_id, + bigtable_instance, + ) + max_versions_rule = column_family.MaxVersionsGCRule(1) + column_family_id = "body_mass_g" + column_families = {column_family_id: max_versions_rule} + bt_table.create(column_families=column_families) + yield bt_table + bt_table.delete() + + +@pytest.mark.flaky(retries=3, delay=10) +def test_streaming_df_to_bigtable( + session_load: bigframes.Session, bigtable_table: table.Table +): + # launch a continuous query + job_id_prefix = "test_streaming_" + sdf = session_load.read_gbq_table_streaming("birds.penguins_bigtable_streaming") + + sdf = sdf[["species", "island", "body_mass_g"]] + sdf = sdf[sdf["body_mass_g"] < 4000] + sdf = sdf.rename(columns={"island": "rowkey"}) + + try: + query_job = sdf.to_bigtable( + instance="streaming-testing-instance", + table=bigtable_table.table_id, + service_account_email="streaming-testing-admin@bigframes-load-testing.iam.gserviceaccount.com", + app_profile=None, + truncate=True, + overwrite=True, + auto_create_column_families=True, + bigtable_options={}, + job_id=None, + job_id_prefix=job_id_prefix, + ) + + # wait 100 seconds in order to ensure the query doesn't stop + # (i.e. it is continuous) + time.sleep(100) + assert query_job.running() + assert query_job.error_result is None + assert str(query_job.job_id).startswith(job_id_prefix) + assert len(list(bigtable_table.read_rows())) > 0 + finally: + query_job.cancel() diff --git a/tests/system/large/test_streaming.py b/tests/system/large/streaming/test_pubsub.py similarity index 57% rename from tests/system/large/test_streaming.py rename to tests/system/large/streaming/test_pubsub.py index f80088cf69..277b44c93b 100644 --- a/tests/system/large/test_streaming.py +++ b/tests/system/large/streaming/test_pubsub.py @@ -1,4 +1,4 @@ -# Copyright 2024 Google LLC +# Copyright 2025 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,62 +13,22 @@ # limitations under the License. from concurrent import futures -import time from typing import Generator import uuid -from google.cloud import bigtable, pubsub # type: ignore -from google.cloud.bigtable import column_family, instance, table import pytest import bigframes +pytest.importorskip("google.cloud.pubsub") +from google.cloud import pubsub # type: ignore # noqa + def resource_name_full(project_id: str, resource_type: str, resource_id: str): + """Used for bigtable or pubsub resources.""" return f"projects/{project_id}/{resource_type}/{resource_id}" -@pytest.fixture(scope="session") -def bigtable_instance(session_load: bigframes.Session) -> instance.Instance: - client = bigtable.Client(project=session_load._project, admin=True) - - instance_name = "streaming-testing-instance" - bt_instance = instance.Instance( - instance_name, - client, - ) - - if not bt_instance.exists(): - cluster_id = "streaming-testing-instance-c1" - cluster = bt_instance.cluster( - cluster_id, - location_id="us-west1-a", - serve_nodes=1, - ) - operation = bt_instance.create( - clusters=[cluster], - ) - operation.result(timeout=480) - return bt_instance - - -@pytest.fixture(scope="function") -def bigtable_table( - bigtable_instance: instance.Instance, -) -> Generator[table.Table, None, None]: - table_id = "bigframes_test_" + uuid.uuid4().hex - bt_table = table.Table( - table_id, - bigtable_instance, - ) - max_versions_rule = column_family.MaxVersionsGCRule(1) - column_family_id = "body_mass_g" - column_families = {column_family_id: max_versions_rule} - bt_table.create(column_families=column_families) - yield bt_table - bt_table.delete() - - @pytest.fixture(scope="function") def pubsub_topic_id(session_load: bigframes.Session) -> Generator[str, None, None]: publisher = pubsub.PublisherClient() @@ -98,43 +58,6 @@ def pubsub_topic_subscription_ids( subscriber.delete_subscription(subscription=subscription_name) -@pytest.mark.flaky(retries=3, delay=10) -def test_streaming_df_to_bigtable( - session_load: bigframes.Session, bigtable_table: table.Table -): - # launch a continuous query - job_id_prefix = "test_streaming_" - sdf = session_load.read_gbq_table_streaming("birds.penguins_bigtable_streaming") - - sdf = sdf[["species", "island", "body_mass_g"]] - sdf = sdf[sdf["body_mass_g"] < 4000] - sdf = sdf.rename(columns={"island": "rowkey"}) - - try: - query_job = sdf.to_bigtable( - instance="streaming-testing-instance", - table=bigtable_table.table_id, - service_account_email="streaming-testing-admin@bigframes-load-testing.iam.gserviceaccount.com", - app_profile=None, - truncate=True, - overwrite=True, - auto_create_column_families=True, - bigtable_options={}, - job_id=None, - job_id_prefix=job_id_prefix, - ) - - # wait 100 seconds in order to ensure the query doesn't stop - # (i.e. it is continuous) - time.sleep(100) - assert query_job.running() - assert query_job.error_result is None - assert str(query_job.job_id).startswith(job_id_prefix) - assert len(list(bigtable_table.read_rows())) > 0 - finally: - query_job.cancel() - - @pytest.mark.flaky(retries=3, delay=10) def test_streaming_df_to_pubsub( session_load: bigframes.Session, pubsub_topic_subscription_ids: tuple[str, str] From 599f18be0d1f5f892fc1c61f5d4561ae1e3cb5bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tim=20Swe=C3=B1a=20=28Swast=29?= Date: Thu, 8 May 2025 10:58:05 -0500 Subject: [PATCH 18/36] chore: create private `_read_gbq_colab` which will enable partial ordering mode even when session is in strict mode (#1688) * chore: add private _read_gbq_colab method that uses partial ordering mode, disables progress bars, disables default index, and communicates via callbacks * add colab read gbq * add test for ordering * add ordered argument to to_pandas_batches * add unit test looking for job labels * remove ordered option for to_pandas_batches * ignore type for mock job configs --- bigframes/session/__init__.py | 21 ++++++++++ bigframes/session/loader.py | 16 +++++++- bigframes/testing/mocks.py | 16 ++++++-- tests/system/small/session/__init__.py | 13 ++++++ .../small/session/test_read_gbq_colab.py | 41 +++++++++++++++++++ tests/unit/session/test_read_gbq_colab.py | 32 +++++++++++++++ 6 files changed, 135 insertions(+), 4 deletions(-) create mode 100644 tests/system/small/session/__init__.py create mode 100644 tests/system/small/session/test_read_gbq_colab.py create mode 100644 tests/unit/session/test_read_gbq_colab.py diff --git a/bigframes/session/__init__.py b/bigframes/session/__init__.py index 1081270c76..7260553c14 100644 --- a/bigframes/session/__init__.py +++ b/bigframes/session/__init__.py @@ -476,6 +476,27 @@ def _register_object( ): self._objects.append(weakref.ref(object)) + def _read_gbq_colab( + self, + query: str, + # TODO: Add a callback parameter that takes some kind of Event object. + # TODO: Add parameter for variables for string formatting. + # TODO: Add dry_run parameter. + ) -> dataframe.DataFrame: + """A version of read_gbq that has the necessary default values for use in colab integrations. + + This includes, no ordering, no index, no progress bar, always use string + formatting for embedding local variables / dataframes. + """ + + # TODO: Allow for a table ID to avoid queries like read_gbq? + return self._loader.read_gbq_query( + query=query, + index_col=bigframes.enums.DefaultIndexKind.NULL, + api_name="read_gbq_colab", + force_total_order=False, + ) + @overload def read_gbq_query( # type: ignore[overload-overlap] self, diff --git a/bigframes/session/loader.py b/bigframes/session/loader.py index d9adb8683a..b630dedb7b 100644 --- a/bigframes/session/loader.py +++ b/bigframes/session/loader.py @@ -373,6 +373,7 @@ def read_gbq_table( # type: ignore[overload-overlap] filters: third_party_pandas_gbq.FiltersType = ..., enable_snapshot: bool = ..., dry_run: Literal[False] = ..., + force_total_order: Optional[bool] = ..., ) -> dataframe.DataFrame: ... @@ -394,6 +395,7 @@ def read_gbq_table( filters: third_party_pandas_gbq.FiltersType = ..., enable_snapshot: bool = ..., dry_run: Literal[True] = ..., + force_total_order: Optional[bool] = ..., ) -> pandas.Series: ... @@ -414,6 +416,7 @@ def read_gbq_table( filters: third_party_pandas_gbq.FiltersType = (), enable_snapshot: bool = True, dry_run: bool = False, + force_total_order: Optional[bool] = None, ) -> dataframe.DataFrame | pandas.Series: import bigframes._tools.strings import bigframes.dataframe as dataframe @@ -608,7 +611,14 @@ def read_gbq_table( session=self._session, ) # if we don't have a unique index, we order by row hash if we are in strict mode - if self._force_total_order: + if ( + # If the user has explicitly selected or disabled total ordering for + # this API call, respect that choice. + (force_total_order is not None and force_total_order) + # If the user has not explicitly selected or disabled total ordering + # for this API call, respect the default choice for the session. + or (force_total_order is None and self._force_total_order) + ): if not primary_key: array_value = array_value.order_by( [ @@ -712,6 +722,7 @@ def read_gbq_query( # type: ignore[overload-overlap] use_cache: Optional[bool] = ..., filters: third_party_pandas_gbq.FiltersType = ..., dry_run: Literal[False] = ..., + force_total_order: Optional[bool] = ..., ) -> dataframe.DataFrame: ... @@ -728,6 +739,7 @@ def read_gbq_query( use_cache: Optional[bool] = ..., filters: third_party_pandas_gbq.FiltersType = ..., dry_run: Literal[True] = ..., + force_total_order: Optional[bool] = ..., ) -> pandas.Series: ... @@ -743,6 +755,7 @@ def read_gbq_query( use_cache: Optional[bool] = None, filters: third_party_pandas_gbq.FiltersType = (), dry_run: bool = False, + force_total_order: Optional[bool] = None, ) -> dataframe.DataFrame | pandas.Series: import bigframes.dataframe as dataframe @@ -833,6 +846,7 @@ def read_gbq_query( columns=columns, use_cache=configuration["query"]["useQueryCache"], api_name=api_name, + force_total_order=force_total_order, # max_results and filters are omitted because they are already # handled by to_query(), above. ) diff --git a/bigframes/testing/mocks.py b/bigframes/testing/mocks.py index ab48b97f0d..d0cfb50ad5 100644 --- a/bigframes/testing/mocks.py +++ b/bigframes/testing/mocks.py @@ -51,6 +51,9 @@ def create_bigquery_session( google.auth.credentials.Credentials, instance=True ) + bq_time = datetime.datetime.now() + table_time = bq_time + datetime.timedelta(minutes=1) + if anonymous_dataset is None: anonymous_dataset = google.cloud.bigquery.DatasetReference( "test-project", @@ -65,6 +68,8 @@ def create_bigquery_session( # Mock the location. table = mock.create_autospec(google.cloud.bigquery.Table, instance=True) table._properties = {} + # TODO(tswast): support tables created before and after the session started. + type(table).created = mock.PropertyMock(return_value=table_time) type(table).location = mock.PropertyMock(return_value=location) type(table).schema = mock.PropertyMock(return_value=table_schema) type(table).reference = mock.PropertyMock( @@ -73,7 +78,10 @@ def create_bigquery_session( type(table).num_rows = mock.PropertyMock(return_value=1000000000) bqclient.get_table.return_value = table - def query_mock(query, *args, **kwargs): + job_configs = [] + + def query_mock(query, *args, job_config=None, **kwargs): + job_configs.append(job_config) query_job = mock.create_autospec(google.cloud.bigquery.QueryJob) type(query_job).destination = mock.PropertyMock( return_value=anonymous_dataset.table("test_table"), @@ -83,7 +91,7 @@ def query_mock(query, *args, **kwargs): ) if query.startswith("SELECT CURRENT_TIMESTAMP()"): - query_job.result = mock.MagicMock(return_value=[[datetime.datetime.now()]]) + query_job.result = mock.MagicMock(return_value=[[bq_time]]) else: type(query_job).schema = mock.PropertyMock(return_value=table_schema) @@ -91,7 +99,8 @@ def query_mock(query, *args, **kwargs): existing_query_and_wait = bqclient.query_and_wait - def query_and_wait_mock(query, *args, **kwargs): + def query_and_wait_mock(query, *args, job_config=None, **kwargs): + job_configs.append(job_config) if query.startswith("SELECT CURRENT_TIMESTAMP()"): return iter([[datetime.datetime.now()]]) else: @@ -109,6 +118,7 @@ def query_and_wait_mock(query, *args, **kwargs): session._bq_connection_manager = mock.create_autospec( bigframes.clients.BqConnectionManager, instance=True ) + session._job_configs = job_configs # type: ignore return session diff --git a/tests/system/small/session/__init__.py b/tests/system/small/session/__init__.py new file mode 100644 index 0000000000..0a2669d7a2 --- /dev/null +++ b/tests/system/small/session/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. diff --git a/tests/system/small/session/test_read_gbq_colab.py b/tests/system/small/session/test_read_gbq_colab.py new file mode 100644 index 0000000000..7ade85b2b2 --- /dev/null +++ b/tests/system/small/session/test_read_gbq_colab.py @@ -0,0 +1,41 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +"""System tests for read_gbq_colab helper functions.""" + + +def test_read_gbq_colab_to_pandas_batches_preserves_order_by(maybe_ordered_session): + df = maybe_ordered_session._read_gbq_colab( + """ + SELECT + name, + SUM(number) AS total + FROM + `bigquery-public-data.usa_names.usa_1910_2013` + WHERE state LIKE 'W%' + GROUP BY name + ORDER BY total DESC + LIMIT 300 + """ + ) + batches = df.to_pandas_batches( + page_size=100, + ) + + total_rows = 0 + for batch in batches: + assert batch["total"].is_monotonic_decreasing + total_rows += len(batch.index) + + assert total_rows > 0 diff --git a/tests/unit/session/test_read_gbq_colab.py b/tests/unit/session/test_read_gbq_colab.py new file mode 100644 index 0000000000..ddca220a79 --- /dev/null +++ b/tests/unit/session/test_read_gbq_colab.py @@ -0,0 +1,32 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +"""Unit tests for read_gbq_colab helper functions.""" + +from bigframes.testing import mocks + + +def test_read_gbq_colab_includes_label(): + """Make sure we can tell direct colab usage apart from regular read_gbq usage.""" + session = mocks.create_bigquery_session() + _ = session._read_gbq_colab("SELECT 'read-gbq-colab-test'") + configs = session._job_configs # type: ignore + + label_values = [] + for config in configs: + if config is None: + continue + label_values.extend(config.labels.values()) + + assert "read_gbq_colab" in label_values From 4389f4c5f10ea05245e56e3886f3c3c82abe56d4 Mon Sep 17 00:00:00 2001 From: Shenyang Cai Date: Thu, 8 May 2025 10:30:30 -0700 Subject: [PATCH 19/36] chore: improve get_started notebook (#1706) * chore: improve get_started notebook * some more polishing * add dots --- .../getting_started_bq_dataframes.ipynb | 840 +++++++----------- 1 file changed, 343 insertions(+), 497 deletions(-) diff --git a/notebooks/getting_started/getting_started_bq_dataframes.ipynb b/notebooks/getting_started/getting_started_bq_dataframes.ipynb index a8158bcb85..ccecd09cb9 100644 --- a/notebooks/getting_started/getting_started_bq_dataframes.ipynb +++ b/notebooks/getting_started/getting_started_bq_dataframes.ipynb @@ -29,7 +29,7 @@ "id": "JAPoU8Sm5E6e" }, "source": [ - "# Get started with BigQuery DataFrames\n", + "# BigQuery DataFrames Quickstart Guide\n", "\n", "\n", "\n", @@ -67,7 +67,7 @@ "source": [ "**_NOTE_**: This notebook has been tested in the following environment:\n", "\n", - "* Python version = 3.10" + "* Python version = 3.12" ] }, { @@ -78,31 +78,17 @@ "source": [ "## Overview\n", "\n", - "Use this notebook to get started with BigQuery DataFrames, including setup, installation, and basic tutorials.\n", - "\n", - "BigQuery DataFrames provides a Pythonic DataFrame and machine learning (ML) API powered by the BigQuery engine.\n", - "\n", - "* `bigframes.pandas` provides a pandas-like API for analytics.\n", - "* `bigframes.ml` provides a scikit-learn-like API for ML.\n", - "\n", - "Learn more about [BigQuery DataFrames](https://cloud.google.com/python/docs/reference/bigframes/latest)." - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "d975e698c9a4" - }, - "source": [ - "### Objective\n", - "\n", - "In this tutorial, you learn how to install BigQuery DataFrames, load data into a BigQuery DataFrames DataFrame, and inspect and manipulate the data using pandas and a custom Python function, running at BigQuery scale.\n", + "In this guide, you learn how to install BigQuery DataFrames, load data into a BigQuery DataFrames DataFrame, and inspect and manipulate the data using pandas and a custom Python function, running at BigQuery scale.\n", "\n", "The steps include:\n", "\n", + "- Installing the BigQuery Dataframes package.\n", + "- Setting up the environment.\n", "- Creating a BigQuery DataFrames DataFrame: Access data from a local CSV to create a BigQuery DataFrames DataFrame.\n", "- Inspecting and manipulating data: Use pandas to perform data cleaning and preparation on the DataFrame.\n", - "- Deploying a custom function: Deploy a [remote function ](https://cloud.google.com/bigquery/docs/remote-functions)that runs a scalar Python function at BigQuery scale." + "- Deploying a custom function: Deploy a [remote function ](https://cloud.google.com/bigquery/docs/remote-functions)that runs a scalar Python function at BigQuery scale.\n", + "\n", + "You can learn more about [BigQuery DataFrames](https://cloud.google.com/python/docs/reference/bigframes/latest)." ] }, { @@ -151,122 +137,11 @@ }, { "cell_type": "code", - "execution_count": 2, + "execution_count": null, "metadata": { "id": "mfPoOwPLGpSr" }, - "outputs": [ - { - "name": "stdout", - "output_type": "stream", - "text": [ - "Requirement already satisfied: bigframes in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (0.25.0)\n", - "Requirement already satisfied: cloudpickle>=2.0.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (3.0.0)\n", - "Requirement already satisfied: fsspec>=2023.3.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2024.2.0)\n", - "Requirement already satisfied: gcsfs>=2023.3.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2024.2.0)\n", - "Requirement already satisfied: geopandas>=0.12.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (0.14.3)\n", - "Requirement already satisfied: google-auth<3.0dev,>=2.15.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2.28.2)\n", - "Requirement already satisfied: google-cloud-bigquery>=3.10.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (3.19.0)\n", - "Requirement already satisfied: google-cloud-functions>=1.12.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (1.16.3)\n", - "Requirement already satisfied: google-cloud-bigquery-connection>=1.12.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (1.15.3)\n", - "Requirement already satisfied: google-cloud-iam>=2.12.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2.14.3)\n", - "Requirement already satisfied: google-cloud-resource-manager>=1.10.3 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (1.12.3)\n", - "Requirement already satisfied: google-cloud-storage>=2.0.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2.15.0)\n", - "Requirement already satisfied: ibis-framework<9.0.0dev,>=8.0.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (8.0.0)\n", - "Requirement already satisfied: pandas<2.1.4,>=1.5.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2.1.3)\n", - "Requirement already satisfied: pydata-google-auth>=1.8.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (1.8.2)\n", - "Requirement already satisfied: requests>=2.27.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2.31.0)\n", - "Requirement already satisfied: scikit-learn>=1.2.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (1.4.1.post1)\n", - "Requirement already satisfied: sqlalchemy<3.0dev,>=1.4 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (2.0.28)\n", - "Requirement already satisfied: sqlglot<=20.11,>=20.8.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (20.11.0)\n", - "Requirement already satisfied: tabulate>=0.9 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (0.9.0)\n", - "Requirement already satisfied: ipywidgets>=7.7.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (8.1.2)\n", - "Requirement already satisfied: humanize>=4.6.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (4.9.0)\n", - "Requirement already satisfied: matplotlib>=3.7.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from bigframes) (3.8.3)\n", - "Requirement already satisfied: aiohttp!=4.0.0a0,!=4.0.0a1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from gcsfs>=2023.3.0->bigframes) (3.9.3)\n", - "Requirement already satisfied: decorator>4.1.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from gcsfs>=2023.3.0->bigframes) (5.1.1)\n", - "Requirement already satisfied: google-auth-oauthlib in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from gcsfs>=2023.3.0->bigframes) (1.2.0)\n", - "Requirement already satisfied: fiona>=1.8.21 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from geopandas>=0.12.2->bigframes) (1.9.6)\n", - "Requirement already satisfied: packaging in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from geopandas>=0.12.2->bigframes) (24.0)\n", - "Requirement already satisfied: pyproj>=3.3.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from geopandas>=0.12.2->bigframes) (3.6.1)\n", - "Requirement already satisfied: shapely>=1.8.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from geopandas>=0.12.2->bigframes) (2.0.3)\n", - "Requirement already satisfied: cachetools<6.0,>=2.0.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-auth<3.0dev,>=2.15.0->bigframes) (5.3.3)\n", - "Requirement already satisfied: pyasn1-modules>=0.2.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-auth<3.0dev,>=2.15.0->bigframes) (0.3.0)\n", - "Requirement already satisfied: rsa<5,>=3.1.4 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-auth<3.0dev,>=2.15.0->bigframes) (4.9)\n", - "Requirement already satisfied: google-api-core!=2.0.*,!=2.1.*,!=2.10.*,!=2.2.*,!=2.3.*,!=2.4.*,!=2.5.*,!=2.6.*,!=2.7.*,!=2.8.*,!=2.9.*,<3.0.0dev,>=1.34.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-api-core[grpc]!=2.0.*,!=2.1.*,!=2.10.*,!=2.2.*,!=2.3.*,!=2.4.*,!=2.5.*,!=2.6.*,!=2.7.*,!=2.8.*,!=2.9.*,<3.0.0dev,>=1.34.1->google-cloud-bigquery>=3.10.0->google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (2.17.1)\n", - "Requirement already satisfied: google-cloud-core<3.0.0dev,>=1.6.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery>=3.10.0->google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (2.4.1)\n", - "Requirement already satisfied: google-resumable-media<3.0dev,>=0.6.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery>=3.10.0->google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (2.7.0)\n", - "Requirement already satisfied: python-dateutil<3.0dev,>=2.7.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery>=3.10.0->google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (2.9.0.post0)\n", - "Requirement already satisfied: proto-plus<2.0.0dev,>=1.22.3 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery-connection>=1.12.0->bigframes) (1.23.0)\n", - "Requirement already satisfied: protobuf!=3.20.0,!=3.20.1,!=4.21.0,!=4.21.1,!=4.21.2,!=4.21.3,!=4.21.4,!=4.21.5,<5.0.0dev,>=3.19.5 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery-connection>=1.12.0->bigframes) (4.25.3)\n", - "Requirement already satisfied: grpc-google-iam-v1<1.0.0dev,>=0.12.4 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery-connection>=1.12.0->bigframes) (0.13.0)\n", - "Requirement already satisfied: google-cloud-bigquery-storage<3.0.0dev,>=2.6.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (2.24.0)\n", - "Requirement already satisfied: grpcio<2.0dev,>=1.47.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (1.62.1)\n", - "Requirement already satisfied: pyarrow>=3.0.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (15.0.1)\n", - "Requirement already satisfied: db-dtypes<2.0.0dev,>=0.3.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (1.2.0)\n", - "Requirement already satisfied: google-crc32c<2.0dev,>=1.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-cloud-storage>=2.0.0->bigframes) (1.5.0)\n", - "Requirement already satisfied: atpublic<5,>=2.3 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (4.0)\n", - "Requirement already satisfied: bidict<1,>=0.22.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (0.23.1)\n", - "Requirement already satisfied: multipledispatch<2,>=0.6 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (1.0.0)\n", - "Requirement already satisfied: numpy<2,>=1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (1.26.4)\n", - "Requirement already satisfied: parsy<3,>=2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (2.1)\n", - "Requirement already satisfied: pyarrow-hotfix<1,>=0.4 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (0.6)\n", - "Requirement already satisfied: pytz>=2022.7 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (2024.1)\n", - "Requirement already satisfied: rich<14,>=12.4.4 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (13.7.1)\n", - "Requirement already satisfied: toolz<1,>=0.11 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (0.12.1)\n", - "Requirement already satisfied: typing-extensions<5,>=4.3.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (4.10.0)\n", - "Requirement already satisfied: comm>=0.1.3 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipywidgets>=7.7.1->bigframes) (0.2.2)\n", - "Requirement already satisfied: ipython>=6.1.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipywidgets>=7.7.1->bigframes) (8.22.2)\n", - "Requirement already satisfied: traitlets>=4.3.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipywidgets>=7.7.1->bigframes) (5.14.2)\n", - "Requirement already satisfied: widgetsnbextension~=4.0.10 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipywidgets>=7.7.1->bigframes) (4.0.10)\n", - "Requirement already satisfied: jupyterlab-widgets~=3.0.10 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipywidgets>=7.7.1->bigframes) (3.0.10)\n", - "Requirement already satisfied: contourpy>=1.0.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from matplotlib>=3.7.1->bigframes) (1.2.0)\n", - "Requirement already satisfied: cycler>=0.10 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from matplotlib>=3.7.1->bigframes) (0.12.1)\n", - "Requirement already satisfied: fonttools>=4.22.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from matplotlib>=3.7.1->bigframes) (4.49.0)\n", - "Requirement already satisfied: kiwisolver>=1.3.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from matplotlib>=3.7.1->bigframes) (1.4.5)\n", - "Requirement already satisfied: pillow>=8 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from matplotlib>=3.7.1->bigframes) (10.2.0)\n", - "Requirement already satisfied: pyparsing>=2.3.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from matplotlib>=3.7.1->bigframes) (3.1.2)\n", - "Requirement already satisfied: tzdata>=2022.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from pandas<2.1.4,>=1.5.0->bigframes) (2024.1)\n", - "Requirement already satisfied: setuptools in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from pydata-google-auth>=1.8.2->bigframes) (69.2.0)\n", - "Requirement already satisfied: charset-normalizer<4,>=2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from requests>=2.27.1->bigframes) (3.3.2)\n", - "Requirement already satisfied: idna<4,>=2.5 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from requests>=2.27.1->bigframes) (3.6)\n", - "Requirement already satisfied: urllib3<3,>=1.21.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from requests>=2.27.1->bigframes) (2.2.1)\n", - "Requirement already satisfied: certifi>=2017.4.17 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from requests>=2.27.1->bigframes) (2024.2.2)\n", - "Requirement already satisfied: scipy>=1.6.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from scikit-learn>=1.2.2->bigframes) (1.12.0)\n", - "Requirement already satisfied: joblib>=1.2.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from scikit-learn>=1.2.2->bigframes) (1.3.2)\n", - "Requirement already satisfied: threadpoolctl>=2.0.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from scikit-learn>=1.2.2->bigframes) (3.3.0)\n", - "Requirement already satisfied: greenlet!=0.4.17 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from sqlalchemy<3.0dev,>=1.4->bigframes) (3.0.3)\n", - "Requirement already satisfied: aiosignal>=1.1.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from aiohttp!=4.0.0a0,!=4.0.0a1->gcsfs>=2023.3.0->bigframes) (1.3.1)\n", - "Requirement already satisfied: attrs>=17.3.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from aiohttp!=4.0.0a0,!=4.0.0a1->gcsfs>=2023.3.0->bigframes) (23.2.0)\n", - "Requirement already satisfied: frozenlist>=1.1.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from aiohttp!=4.0.0a0,!=4.0.0a1->gcsfs>=2023.3.0->bigframes) (1.4.1)\n", - "Requirement already satisfied: multidict<7.0,>=4.5 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from aiohttp!=4.0.0a0,!=4.0.0a1->gcsfs>=2023.3.0->bigframes) (6.0.5)\n", - "Requirement already satisfied: yarl<2.0,>=1.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from aiohttp!=4.0.0a0,!=4.0.0a1->gcsfs>=2023.3.0->bigframes) (1.9.4)\n", - "Requirement already satisfied: click~=8.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from fiona>=1.8.21->geopandas>=0.12.2->bigframes) (8.1.7)\n", - "Requirement already satisfied: click-plugins>=1.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from fiona>=1.8.21->geopandas>=0.12.2->bigframes) (1.1.1)\n", - "Requirement already satisfied: cligj>=0.5 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from fiona>=1.8.21->geopandas>=0.12.2->bigframes) (0.7.2)\n", - "Requirement already satisfied: six in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from fiona>=1.8.21->geopandas>=0.12.2->bigframes) (1.16.0)\n", - "Requirement already satisfied: googleapis-common-protos<2.0.dev0,>=1.56.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-api-core!=2.0.*,!=2.1.*,!=2.10.*,!=2.2.*,!=2.3.*,!=2.4.*,!=2.5.*,!=2.6.*,!=2.7.*,!=2.8.*,!=2.9.*,<3.0.0dev,>=1.34.1->google-api-core[grpc]!=2.0.*,!=2.1.*,!=2.10.*,!=2.2.*,!=2.3.*,!=2.4.*,!=2.5.*,!=2.6.*,!=2.7.*,!=2.8.*,!=2.9.*,<3.0.0dev,>=1.34.1->google-cloud-bigquery>=3.10.0->google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (1.63.0)\n", - "Requirement already satisfied: grpcio-status<2.0.dev0,>=1.33.2 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-api-core[grpc]!=2.0.*,!=2.1.*,!=2.10.*,!=2.2.*,!=2.3.*,!=2.4.*,!=2.5.*,!=2.6.*,!=2.7.*,!=2.8.*,!=2.9.*,<3.0.0dev,>=1.34.1->google-cloud-bigquery>=3.10.0->google-cloud-bigquery[bqstorage,pandas]>=3.10.0->bigframes) (1.62.1)\n", - "Requirement already satisfied: requests-oauthlib>=0.7.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from google-auth-oauthlib->gcsfs>=2023.3.0->bigframes) (1.4.0)\n", - "Requirement already satisfied: jedi>=0.16 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (0.19.1)\n", - "Requirement already satisfied: matplotlib-inline in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (0.1.6)\n", - "Requirement already satisfied: prompt-toolkit<3.1.0,>=3.0.41 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (3.0.43)\n", - "Requirement already satisfied: pygments>=2.4.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (2.17.2)\n", - "Requirement already satisfied: stack-data in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (0.6.3)\n", - "Requirement already satisfied: pexpect>4.3 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (4.9.0)\n", - "Requirement already satisfied: pyasn1<0.6.0,>=0.4.6 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from pyasn1-modules>=0.2.1->google-auth<3.0dev,>=2.15.0->bigframes) (0.5.1)\n", - "Requirement already satisfied: markdown-it-py>=2.2.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from rich<14,>=12.4.4->ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (3.0.0)\n", - "Requirement already satisfied: parso<0.9.0,>=0.8.3 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from jedi>=0.16->ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (0.8.3)\n", - "Requirement already satisfied: mdurl~=0.1 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from markdown-it-py>=2.2.0->rich<14,>=12.4.4->ibis-framework<9.0.0dev,>=8.0.0->ibis-framework[bigquery]<9.0.0dev,>=8.0.0->bigframes) (0.1.2)\n", - "Requirement already satisfied: ptyprocess>=0.5 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from pexpect>4.3->ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (0.7.0)\n", - "Requirement already satisfied: wcwidth in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from prompt-toolkit<3.1.0,>=3.0.41->ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (0.2.13)\n", - "Requirement already satisfied: oauthlib>=3.0.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from requests-oauthlib>=0.7.0->google-auth-oauthlib->gcsfs>=2023.3.0->bigframes) (3.2.2)\n", - "Requirement already satisfied: executing>=1.2.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from stack-data->ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (2.0.1)\n", - "Requirement already satisfied: asttokens>=2.1.0 in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from stack-data->ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (2.4.1)\n", - "Requirement already satisfied: pure-eval in /usr/local/google/home/swast/envs/bigframes/lib/python3.11/site-packages (from stack-data->ipython>=6.1.0->ipywidgets>=7.7.1->bigframes) (0.2.2)\n" - ] - } - ], + "outputs": [], "source": [ "!pip install bigframes" ] @@ -303,7 +178,7 @@ "id": "BF1j6f9HApxa" }, "source": [ - "## Before you begin\n", + "## Environment setup\n", "\n", "Complete the tasks in this section to set up your environment." ] @@ -351,7 +226,7 @@ }, { "cell_type": "code", - "execution_count": 4, + "execution_count": null, "metadata": { "id": "oM1iC_MfAts1" }, @@ -360,12 +235,7 @@ "name": "stdout", "output_type": "stream", "text": [ - "Updated property [core/project].\n", - "\n", - "\n", - "To take a quick anonymous survey, run:\n", - " $ gcloud survey\n", - "\n" + "Updated property [core/project].\n" ] } ], @@ -558,11 +428,24 @@ }, { "cell_type": "code", - "execution_count": 10, + "execution_count": null, "metadata": { "id": "Vyex9BQI-BNa" }, - "outputs": [], + "outputs": [ + { + "data": { + "text/html": [ + "Query job badadf0b-27c8-4dac-a468-be3c40745538 is DONE. 0 Bytes processed. Open Job" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], "source": [ "# bq_df_sample = bpd.read_gbq(\"bigquery-samples.wikipedia_pageviews.200809h\")" ] @@ -584,21 +467,135 @@ "source": [ "Uncomment and run the following cell to see pandas in action over your new BigQuery DataFrames DataFrame.\n", "\n", - "This code uses regex to filter the DataFrame to include only rows with Wikipedia page titles containing the word \"Google\", sums the total views by page title, and then returns the top 100 results." + "This code uses regex to filter the DataFrame to include only rows with Wikipedia page titles containing the word \"Google\", sums the total views by page title, and then returns the top 10 results." ] }, { "cell_type": "code", - "execution_count": 11, + "execution_count": null, "metadata": { "id": "XfGq5apK-D_e" }, - "outputs": [], + "outputs": [ + { + "data": { + "text/html": [ + "Query job c8669c7f-bca3-4f54-b354-8e57b3321f5a is DONE. 34.9 GB processed. Open Job" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
titleviews
21911Google1414560
27669Google_Chrome962482
28394Google_Earth383566
29184Google_Maps205089
27251Google_Android99450
33900Google_search97665
31825Google_chrome78399
30204Google_Street_View71580
40798Image:Google_Chrome.png60746
35222Googleplex53848
\n", + "

10 rows × 2 columns

\n", + "[10 rows x 2 columns in total]" + ], + "text/plain": [ + " title views\n", + "21911 Google 1414560\n", + "27669 Google_Chrome 962482\n", + "28394 Google_Earth 383566\n", + "29184 Google_Maps 205089\n", + "27251 Google_Android 99450\n", + "33900 Google_search 97665\n", + "31825 Google_chrome 78399\n", + "30204 Google_Street_View 71580\n", + "40798 Image:Google_Chrome.png 60746\n", + "35222 Googleplex 53848\n", + "\n", + "[10 rows x 2 columns]" + ] + }, + "execution_count": 11, + "metadata": {}, + "output_type": "execute_result" + } + ], "source": [ "# bq_df_sample[bq_df_sample.title.str.contains(r\"[Gg]oogle\")]\\\n", - "# .groupby(['title'], as_index=False)['views'].sum(numeric_only=True)\\\n", - "# .sort_values('views', ascending=False)\\\n", - "# .head(100)" + "# .groupby(['title'], as_index=False)['views'].sum(numeric_only=True)\\\n", + "# .sort_values('views', ascending=False)\\\n", + "# .head(10)" ] }, { @@ -723,7 +720,7 @@ { "data": { "text/html": [ - "Load job d578c399-e2e5-4f6b-ba28-59d0686a91e7 is DONE. Open Job" + "Load job 93903930-10b8-48b8-b41b-3da54917b281 is DONE. Open Job" ], "text/plain": [ "" @@ -745,7 +742,7 @@ "id": "U-RVfNCu_h_h" }, "source": [ - "Take a look at the first few rows of the DataFrame:" + "Take a look at the rows randomly sampled from the DataFrame:" ] }, { @@ -758,31 +755,7 @@ { "data": { "text/html": [ - "Query job f50a129b-4a51-4c21-b155-ab1e85c1403e is DONE. 28.9 kB processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job b0d65008-f9f1-4fec-8620-42f307390049 is DONE. 0 Bytes processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job a83d72e8-0cb8-44e9-ad0b-6fe3726ed1e9 is DONE. 501 Bytes processed. Open Job" + "Query job 17f58b5c-88b2-4b26-8d0d-cc3d9a979a06 is DONE. 28.9 kB processed. Open Job" ], "text/plain": [ "" @@ -823,76 +796,73 @@ " \n", " \n", " \n", - " 0\n", - " Gentoo penguin (Pygoscelis papua)\n", + " 78\n", + " Chinstrap penguin (Pygoscelis antarctica)\n", + " Dream\n", + " 47.0\n", + " 17.3\n", + " 185\n", + " 3700\n", + " FEMALE\n", + " \n", + " \n", + " 130\n", + " Adelie Penguin (Pygoscelis adeliae)\n", " Biscoe\n", - " 50.5\n", - " 15.9\n", - " 225\n", - " 5400\n", - " MALE\n", + " 40.5\n", + " 17.9\n", + " 187\n", + " 3200\n", + " FEMALE\n", " \n", " \n", - " 1\n", + " 84\n", " Gentoo penguin (Pygoscelis papua)\n", " Biscoe\n", - " 45.1\n", + " 49.1\n", " 14.5\n", - " 215\n", - " 5000\n", + " 212\n", + " 4625\n", " FEMALE\n", " \n", " \n", - " 2\n", + " 334\n", " Adelie Penguin (Pygoscelis adeliae)\n", - " Torgersen\n", - " 41.4\n", - " 18.5\n", - " 202\n", - " 3875\n", + " Biscoe\n", + " 38.2\n", + " 20.0\n", + " 190\n", + " 3900\n", " MALE\n", " \n", " \n", - " 3\n", - " Adelie Penguin (Pygoscelis adeliae)\n", - " Torgersen\n", - " 38.6\n", - " 17.0\n", - " 188\n", - " 2900\n", - " FEMALE\n", - " \n", - " \n", - " 4\n", - " Gentoo penguin (Pygoscelis papua)\n", - " Biscoe\n", - " 46.5\n", - " 14.8\n", - " 217\n", - " 5200\n", - " FEMALE\n", + " 67\n", + " Chinstrap penguin (Pygoscelis antarctica)\n", + " Dream\n", + " 55.8\n", + " 19.8\n", + " 207\n", + " 4000\n", + " MALE\n", " \n", " \n", "\n", - "

5 rows × 7 columns

\n", - "[5 rows x 7 columns in total]" + "" ], "text/plain": [ - " species island culmen_length_mm \\\n", - "0 Gentoo penguin (Pygoscelis papua) Biscoe 50.5 \n", - "1 Gentoo penguin (Pygoscelis papua) Biscoe 45.1 \n", - "2 Adelie Penguin (Pygoscelis adeliae) Torgersen 41.4 \n", - "3 Adelie Penguin (Pygoscelis adeliae) Torgersen 38.6 \n", - "4 Gentoo penguin (Pygoscelis papua) Biscoe 46.5 \n", - "\n", - " culmen_depth_mm flipper_length_mm body_mass_g sex \n", - "0 15.9 225 5400 MALE \n", - "1 14.5 215 5000 FEMALE \n", - "2 18.5 202 3875 MALE \n", - "3 17.0 188 2900 FEMALE \n", - "4 14.8 217 5200 FEMALE \n", + " species island culmen_length_mm \\\n", + "78 Chinstrap penguin (Pygoscelis antarctica) Dream 47.0 \n", + "130 Adelie Penguin (Pygoscelis adeliae) Biscoe 40.5 \n", + "84 Gentoo penguin (Pygoscelis papua) Biscoe 49.1 \n", + "334 Adelie Penguin (Pygoscelis adeliae) Biscoe 38.2 \n", + "67 Chinstrap penguin (Pygoscelis antarctica) Dream 55.8 \n", "\n", - "[5 rows x 7 columns]" + " culmen_depth_mm flipper_length_mm body_mass_g sex \n", + "78 17.3 185 3700 FEMALE \n", + "130 17.9 187 3200 FEMALE \n", + "84 14.5 212 4625 FEMALE \n", + "334 20.0 190 3900 MALE \n", + "67 19.8 207 4000 MALE " ] }, "execution_count": 15, @@ -901,7 +871,7 @@ } ], "source": [ - "df_from_local.head()" + "df_from_local.peek()" ] }, { @@ -969,7 +939,7 @@ { "data": { "text/html": [ - "Query job 49702108-948c-4a60-a66e-16a3ed6bc102 is DONE. 28.9 kB processed. Open Job" + "Query job 55aa9cc4-29b6-4052-aae4-5499dc5f1168 is DONE. 28.9 kB processed. Open Job" ], "text/plain": [ "" @@ -981,7 +951,7 @@ { "data": { "text/plain": [ - "'swast-scratch.birds.penguins'" + "'bigframes-dev.birds.penguins'" ] }, "execution_count": 17, @@ -991,7 +961,7 @@ ], "source": [ "df_from_local.to_gbq(\n", - " PROJECT_ID + \".\" + DATASET_ID + \".penguins\",\n", + " f\"{PROJECT_ID}.{DATASET_ID}.penguins\",\n", " if_exists=\"replace\",\n", ")" ] @@ -1025,31 +995,7 @@ { "data": { "text/html": [ - "Query job 05a6288d-3774-41d0-9884-6bbb5af28942 is DONE. 28.9 kB processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 45383ce0-0ca1-4c16-9832-739e9d325673 is DONE. 0 Bytes processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 2f672140-ddc6-43b6-b79a-318f29bb9239 is DONE. 501 Bytes processed. Open Job" + "Query job 7b2ff811-1563-4ac4-9d21-69f87e8e85bc is DONE. 28.9 kB processed. Open Job" ], "text/plain": [ "" @@ -1090,76 +1036,73 @@ " \n", " \n", " \n", - " 0\n", + " 12\n", " Gentoo penguin (Pygoscelis papua)\n", " Biscoe\n", - " 50.5\n", - " 15.9\n", - " 225\n", - " 5400\n", - " MALE\n", + " 42.7\n", + " 13.7\n", + " 208\n", + " 3950\n", + " FEMALE\n", " \n", " \n", - " 1\n", + " 24\n", " Gentoo penguin (Pygoscelis papua)\n", " Biscoe\n", - " 45.1\n", - " 14.5\n", - " 215\n", - " 5000\n", - " FEMALE\n", + " 45.0\n", + " 15.4\n", + " 220\n", + " 5050\n", + " MALE\n", " \n", " \n", - " 2\n", + " 62\n", " Adelie Penguin (Pygoscelis adeliae)\n", - " Torgersen\n", - " 41.4\n", - " 18.5\n", - " 202\n", - " 3875\n", + " Dream\n", + " 38.8\n", + " 20.0\n", + " 190\n", + " 3950\n", " MALE\n", " \n", " \n", - " 3\n", - " Adelie Penguin (Pygoscelis adeliae)\n", - " Torgersen\n", - " 38.6\n", - " 17.0\n", - " 188\n", - " 2900\n", + " 123\n", + " Chinstrap penguin (Pygoscelis antarctica)\n", + " Dream\n", + " 42.5\n", + " 17.3\n", + " 187\n", + " 3350\n", " FEMALE\n", " \n", " \n", - " 4\n", - " Gentoo penguin (Pygoscelis papua)\n", - " Biscoe\n", - " 46.5\n", - " 14.8\n", - " 217\n", - " 5200\n", - " FEMALE\n", + " 27\n", + " Adelie Penguin (Pygoscelis adeliae)\n", + " Dream\n", + " 44.1\n", + " 19.7\n", + " 196\n", + " 4400\n", + " MALE\n", " \n", " \n", "\n", - "

5 rows × 7 columns

\n", - "[5 rows x 7 columns in total]" + "" ], "text/plain": [ - " species island culmen_length_mm \\\n", - "0 Gentoo penguin (Pygoscelis papua) Biscoe 50.5 \n", - "1 Gentoo penguin (Pygoscelis papua) Biscoe 45.1 \n", - "2 Adelie Penguin (Pygoscelis adeliae) Torgersen 41.4 \n", - "3 Adelie Penguin (Pygoscelis adeliae) Torgersen 38.6 \n", - "4 Gentoo penguin (Pygoscelis papua) Biscoe 46.5 \n", - "\n", - " culmen_depth_mm flipper_length_mm body_mass_g sex \n", - "0 15.9 225 5400 MALE \n", - "1 14.5 215 5000 FEMALE \n", - "2 18.5 202 3875 MALE \n", - "3 17.0 188 2900 FEMALE \n", - "4 14.8 217 5200 FEMALE \n", + " species island culmen_length_mm \\\n", + "12 Gentoo penguin (Pygoscelis papua) Biscoe 42.7 \n", + "24 Gentoo penguin (Pygoscelis papua) Biscoe 45.0 \n", + "62 Adelie Penguin (Pygoscelis adeliae) Dream 38.8 \n", + "123 Chinstrap penguin (Pygoscelis antarctica) Dream 42.5 \n", + "27 Adelie Penguin (Pygoscelis adeliae) Dream 44.1 \n", "\n", - "[5 rows x 7 columns]" + " culmen_depth_mm flipper_length_mm body_mass_g sex \n", + "12 13.7 208 3950 FEMALE \n", + "24 15.4 220 5050 MALE \n", + "62 20.0 190 3950 MALE \n", + "123 17.3 187 3350 FEMALE \n", + "27 19.7 196 4400 MALE " ] }, "execution_count": 18, @@ -1168,9 +1111,9 @@ } ], "source": [ - "query_or_table = f\"\"\"{PROJECT_ID}.{DATASET_ID}.penguins\"\"\"\n", + "query_or_table = f\"{PROJECT_ID}.{DATASET_ID}.penguins\"\n", "bq_df = bpd.read_gbq(query_or_table)\n", - "bq_df.head()" + "bq_df.peek()" ] }, { @@ -1212,31 +1155,7 @@ { "data": { "text/html": [ - "Query job 5c454fa1-a01b-4e95-b947-6f02554a8461 is DONE. 28.9 kB processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 2cffe5c7-c0c6-4495-ad67-1f5fb55654fd is DONE. 0 Bytes processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 96b4dead-f526-4be3-b24d-5d7aec99eeeb is DONE. 240 Bytes processed. Open Job" + "Query job b396baed-6242-4478-9092-f5e86811b045 is DONE. 31.7 kB processed. Open Job" ], "text/plain": [ "" @@ -1248,16 +1167,16 @@ { "data": { "text/plain": [ - "0 5400\n", - "1 5000\n", - "2 3875\n", - "3 2900\n", - "4 5200\n", - "5 3725\n", - "6 2975\n", - "7 4150\n", - "8 5300\n", - "9 4150\n", + "133 \n", + "279 3150\n", + "34 3400\n", + "96 3600\n", + "18 3800\n", + "208 3950\n", + "310 3175\n", + "64 2850\n", + "118 3550\n", + "2 3075\n", "Name: body_mass_g, dtype: Int64" ] }, @@ -1267,7 +1186,7 @@ } ], "source": [ - "bq_df[\"body_mass_g\"].head(10)" + "bq_df[\"body_mass_g\"].peek(10)" ] }, { @@ -1286,23 +1205,11 @@ "id": "YKwCW7Nsavap" }, "outputs": [ - { - "data": { - "text/html": [ - "Query job 635d000c-14ca-4ecf-bc32-1527821cba28 is DONE. 2.7 kB processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, { "name": "stdout", "output_type": "stream", "text": [ - "average_body_mass: 4201.754385964917\n" + "average_body_mass: 4201.754385964913\n" ] } ], @@ -1330,31 +1237,7 @@ { "data": { "text/html": [ - "Query job d22d8e48-26a0-4cfb-83fc-3e52b834f487 is DONE. 15.6 kB processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 64fff5f3-7106-4003-9241-a9b09afed781 is DONE. 0 Bytes processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job c3d566cc-bed1-4361-96ef-f06956982916 is DONE. 163 Bytes processed. Open Job" + "Query job fef05ee2-9690-41a4-bd35-7cded77310f2 is DONE. 15.6 kB processed. Open Job" ], "text/plain": [ "" @@ -1425,7 +1308,7 @@ } ], "source": [ - "bq_df[[\"species\", \"body_mass_g\"]].groupby(by=bq_df[\"species\"]).mean(numeric_only=True).head()" + "bq_df[[\"species\", \"body_mass_g\"]].groupby(by=bq_df[\"species\"]).mean(numeric_only=True)" ] }, { @@ -1450,7 +1333,7 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 22, "metadata": {}, "outputs": [], "source": [ @@ -1468,7 +1351,7 @@ "id": "zjw8toUbHuRD" }, "source": [ - "Running the cell below creates a custom function using the `remote_function` method. This function categorizes a value into one of two buckets: >= 4000 or <4000.\n", + "Running the cell below creates a custom function using the `remote_function` method. This function categorizes a value into one of two buckets: >= 3500 or <3500.\n", "\n", "> Note: Creating a function requires a [BigQuery connection](https://cloud.google.com/bigquery/docs/remote-functions#create_a_remote_function). This code assumes a pre-created connection named `bigframes-default-connection`. If\n", "the connection is not already created, BigQuery DataFrames attempts to create one assuming the [necessary APIs\n", @@ -1479,17 +1362,30 @@ }, { "cell_type": "code", - "execution_count": 22, + "execution_count": 23, "metadata": { "id": "rSWTOG-vb2Fc" }, - "outputs": [], + "outputs": [ + { + "data": { + "text/html": [ + "Query job c7b6c009-d2c4-4739-a6f8-5ef51e6b1851 is DONE. 0 Bytes processed. Open Job" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], "source": [ "@bpd.remote_function(cloud_function_service_account=\"default\")\n", "def get_bucket(num: float) -> str:\n", - " if not num: return \"NA\"\n", - " boundary = 4000\n", - " return \"at_or_above_4000\" if num >= boundary else \"below_4000\"" + " if not num: return \"NA\"\n", + " boundary = 3500\n", + " return \"at_or_above_3500\" if num >= boundary else \"below_3500\"" ] }, { @@ -1505,7 +1401,7 @@ }, { "cell_type": "code", - "execution_count": 23, + "execution_count": 24, "metadata": { "id": "6ejPXoyEQpWE" }, @@ -1514,8 +1410,8 @@ "name": "stdout", "output_type": "stream", "text": [ - "Cloud Function Name projects/swast-scratch/locations/us-central1/functions/bigframes-71a76285da23f28be467ed16826f7276\n", - "Remote Function Name swast-scratch._63cfa399614a54153cc386c27d6c0c6fdb249f9e.bigframes_71a76285da23f28be467ed16826f7276\n" + "Cloud Function Name projects/bigframes-dev/locations/us-central1/functions/bigframes-sessiondf1983-1d02aa9bc80939ba72e7ff69e37e27c8\n", + "Remote Function Name bigframes-dev._f36a8f778c434a1ec421979eaa3bf562a8561e38.bigframes_sessiondf1983_1d02aa9bc80939ba72e7ff69e37e27c8\n" ] } ], @@ -1537,59 +1433,11 @@ }, { "cell_type": "code", - "execution_count": 24, + "execution_count": 25, "metadata": { "id": "NxSd9WZFcIji" }, "outputs": [ - { - "data": { - "text/html": [ - "Query job 9925acd1-d1e7-4746-90d6-4ce8c2ca30a8 is DONE. 28.9 kB processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 2f10b5cd-80bb-4697-9c61-b7848ce15c81 is DONE. 39.6 kB processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 29266b33-3945-44c0-943b-3d6365b9cc7a is DONE. 0 Bytes processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "Query job 19ecf156-8940-4c02-b20e-3e52e18c7239 is DONE. 396 Bytes processed. Open Job" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, { "data": { "text/html": [ @@ -1617,84 +1465,81 @@ " \n", " \n", " \n", - " 0\n", - " 5400\n", - " at_or_above_4000\n", + " 133\n", + " <NA>\n", + " NA\n", " \n", " \n", - " 1\n", - " 5000\n", - " at_or_above_4000\n", + " 279\n", + " 3150\n", + " below_3500\n", " \n", " \n", - " 2\n", - " 3875\n", - " below_4000\n", + " 34\n", + " 3400\n", + " below_3500\n", " \n", " \n", - " 3\n", - " 2900\n", - " below_4000\n", + " 96\n", + " 3600\n", + " at_or_above_3500\n", " \n", " \n", - " 4\n", - " 5200\n", - " at_or_above_4000\n", + " 18\n", + " 3800\n", + " at_or_above_3500\n", " \n", " \n", - " 5\n", - " 3725\n", - " below_4000\n", + " 208\n", + " 3950\n", + " at_or_above_3500\n", " \n", " \n", - " 6\n", - " 2975\n", - " below_4000\n", + " 310\n", + " 3175\n", + " below_3500\n", " \n", " \n", - " 7\n", - " 4150\n", - " at_or_above_4000\n", + " 64\n", + " 2850\n", + " below_3500\n", " \n", " \n", - " 8\n", - " 5300\n", - " at_or_above_4000\n", + " 118\n", + " 3550\n", + " at_or_above_3500\n", " \n", " \n", - " 9\n", - " 4150\n", - " at_or_above_4000\n", + " 2\n", + " 3075\n", + " below_3500\n", " \n", " \n", "\n", - "

10 rows × 2 columns

\n", - "[10 rows x 2 columns in total]" + "" ], "text/plain": [ - " body_mass_g body_mass_bucket\n", - "0 5400 at_or_above_4000\n", - "1 5000 at_or_above_4000\n", - "2 3875 below_4000\n", - "3 2900 below_4000\n", - "4 5200 at_or_above_4000\n", - "5 3725 below_4000\n", - "6 2975 below_4000\n", - "7 4150 at_or_above_4000\n", - "8 5300 at_or_above_4000\n", - "9 4150 at_or_above_4000\n", - "\n", - "[10 rows x 2 columns]" + " body_mass_g body_mass_bucket\n", + "133 NA\n", + "279 3150 below_3500\n", + "34 3400 below_3500\n", + "96 3600 at_or_above_3500\n", + "18 3800 at_or_above_3500\n", + "208 3950 at_or_above_3500\n", + "310 3175 below_3500\n", + "64 2850 below_3500\n", + "118 3550 at_or_above_3500\n", + "2 3075 below_3500" ] }, - "execution_count": 24, + "execution_count": 25, "metadata": {}, "output_type": "execute_result" } ], "source": [ "bq_df = bq_df.assign(body_mass_bucket=bq_df['body_mass_g'].apply(get_bucket))\n", - "bq_df[['body_mass_g', 'body_mass_bucket']].head(10)" + "bq_df[['body_mass_g', 'body_mass_bucket']].peek(10)" ] }, { @@ -1726,7 +1571,7 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 26, "metadata": {}, "outputs": [], "source": [ @@ -1736,7 +1581,7 @@ }, { "cell_type": "code", - "execution_count": 25, + "execution_count": 27, "metadata": { "id": "sx_vKniMq9ZX" }, @@ -1753,7 +1598,7 @@ }, { "cell_type": "code", - "execution_count": 26, + "execution_count": 28, "metadata": { "id": "_dTCXvCxtPw9" }, @@ -1769,7 +1614,7 @@ }, { "cell_type": "code", - "execution_count": 27, + "execution_count": 29, "metadata": { "id": "EDAIIfcpwNOF" }, @@ -1781,7 +1626,7 @@ }, { "cell_type": "code", - "execution_count": 28, + "execution_count": 30, "metadata": { "id": "QwumLUKmVpuH" }, @@ -1799,7 +1644,8 @@ "toc_visible": true }, "kernelspec": { - "display_name": "Python 3", + "display_name": "venv", + "language": "python", "name": "python3" }, "language_info": { @@ -1812,7 +1658,7 @@ "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", - "version": "3.10.15" + "version": "3.12.10" } }, "nbformat": 4, From 3f79528781abe9bfc122f6f6e26bfa08b029265a Mon Sep 17 00:00:00 2001 From: Garrett Wu <6505921+GarrettWu@users.noreply.github.com> Date: Thu, 8 May 2025 10:46:55 -0700 Subject: [PATCH 20/36] feat: add blob.exif function support (#1703) --- bigframes/blob/_functions.py | 32 +++++++++++++++ bigframes/operations/blob.py | 40 +++++++++++++++++++ .../experimental/multimodal_dataframe.ipynb | 2 +- tests/system/large/blob/test_function.py | 24 +++++++++++ 4 files changed, 97 insertions(+), 1 deletion(-) diff --git a/bigframes/blob/_functions.py b/bigframes/blob/_functions.py index f7f035bff4..f8fdb21946 100644 --- a/bigframes/blob/_functions.py +++ b/bigframes/blob/_functions.py @@ -108,6 +108,38 @@ def udf(self): return self._session.read_gbq_function(udf_name) +def exif_func(src_obj_ref_rt: str) -> str: + import io + import json + + from PIL import ExifTags, Image + import requests + from requests import adapters + + session = requests.Session() + session.mount("https://", adapters.HTTPAdapter(max_retries=3)) + + src_obj_ref_rt_json = json.loads(src_obj_ref_rt) + + src_url = src_obj_ref_rt_json["access_urls"]["read_url"] + + response = session.get(src_url, timeout=30) + bts = response.content + + image = Image.open(io.BytesIO(bts)) + exif_data = image.getexif() + exif_dict = {} + if exif_data: + for tag, value in exif_data.items(): + tag_name = ExifTags.TAGS.get(tag, tag) + exif_dict[tag_name] = value + + return json.dumps(exif_dict) + + +exif_func_def = FunctionDef(exif_func, ["pillow", "requests"]) + + # Blur images. Takes ObjectRefRuntime as JSON string. Outputs ObjectRefRuntime JSON string. def image_blur_func( src_obj_ref_rt: str, dst_obj_ref_rt: str, ksize_x: int, ksize_y: int, ext: str diff --git a/bigframes/operations/blob.py b/bigframes/operations/blob.py index d211c2b918..8da88d1ff8 100644 --- a/bigframes/operations/blob.py +++ b/bigframes/operations/blob.py @@ -300,6 +300,46 @@ def get_runtime_json_str( runtime = self._get_runtime(mode=mode, with_metadata=with_metadata) return runtime._apply_unary_op(ops.ToJSONString()) + def exif( + self, + *, + connection: Optional[str] = None, + max_batching_rows: int = 8192, + container_cpu: Union[float, int] = 0.33, + container_memory: str = "512Mi", + ) -> bigframes.series.Series: + """Extract EXIF data. Now only support image types. + + Args: + connection (str or None, default None): BQ connection used for function internet transactions, and the output blob if "dst" is str. If None, uses default connection of the session. + max_batching_rows (int, default 8,192): Max number of rows per batch send to cloud run to execute the function. + container_cpu (int or float, default 0.33): number of container CPUs. Possible values are [0.33, 8]. Floats larger than 1 are cast to intergers. + container_memory (str, default "512Mi"): container memory size. String of the format . Possible values are from 512Mi to 32Gi. + + Returns: + bigframes.series.Series: JSON series of key-value pairs. + """ + + import bigframes.bigquery as bbq + import bigframes.blob._functions as blob_func + + connection = self._resolve_connection(connection) + df = self.get_runtime_json_str(mode="R").to_frame() + + exif_udf = blob_func.TransformFunction( + blob_func.exif_func_def, + session=self._block.session, + connection=connection, + max_batching_rows=max_batching_rows, + container_cpu=container_cpu, + container_memory=container_memory, + ).udf() + + res = self._df_apply_udf(df, exif_udf) + res = bbq.parse_json(res) + + return res + def image_blur( self, ksize: tuple[int, int], diff --git a/notebooks/experimental/multimodal_dataframe.ipynb b/notebooks/experimental/multimodal_dataframe.ipynb index 4a0cd57a45..05b133a345 100644 --- a/notebooks/experimental/multimodal_dataframe.ipynb +++ b/notebooks/experimental/multimodal_dataframe.ipynb @@ -106,7 +106,7 @@ }, "source": [ "### 1. Create Multimodal DataFrame\n", - "There are several ways to create Multimodal DataFrame. The easiest way is from the wiledcard paths." + "There are several ways to create Multimodal DataFrame. The easiest way is from the wildcard paths." ] }, { diff --git a/tests/system/large/blob/test_function.py b/tests/system/large/blob/test_function.py index 5913df8add..c189d249a7 100644 --- a/tests/system/large/blob/test_function.py +++ b/tests/system/large/blob/test_function.py @@ -51,6 +51,30 @@ def images_output_uris(images_output_folder: str) -> list[str]: ] +def test_blob_exif( + bq_connection: str, + test_session: bigframes.Session, +): + exif_image_df = test_session.from_glob_path( + "gs://bigframes_blob_test/images_exif/*", + name="blob_col", + connection=bq_connection, + ) + + actual = exif_image_df["blob_col"].blob.exif(connection=bq_connection) + expected = bpd.Series( + ['{"ExifOffset": 47, "Make": "MyCamera"}'], + session=test_session, + dtype=dtypes.JSON_DTYPE, + ) + pd.testing.assert_series_equal( + actual.to_pandas(), + expected.to_pandas(), + check_dtype=False, + check_index_type=False, + ) + + def test_blob_image_blur_to_series( images_mm_df: bpd.DataFrame, bq_connection: str, From d1ccb524ea26deac1cf9e481e9d55f9ae166247b Mon Sep 17 00:00:00 2001 From: TrevorBergeron Date: Thu, 8 May 2025 12:12:25 -0700 Subject: [PATCH 21/36] feat: Add inplace arg support to sort methods (#1710) --- bigframes/core/indexes/base.py | 8 ++- bigframes/dataframe.py | 69 +++++++++++++++++-- bigframes/ml/metrics/_metrics.py | 4 +- bigframes/series.py | 60 ++++++++++++++-- tests/system/small/test_dataframe.py | 37 ++++++++++ tests/system/small/test_series.py | 25 +++++++ tests/system/small/test_session.py | 9 +-- .../bigframes_vendored/pandas/core/frame.py | 28 ++++++-- .../bigframes_vendored/pandas/core/series.py | 10 ++- 9 files changed, 226 insertions(+), 24 deletions(-) diff --git a/bigframes/core/indexes/base.py b/bigframes/core/indexes/base.py index 71dc914ed4..eac1f58eae 100644 --- a/bigframes/core/indexes/base.py +++ b/bigframes/core/indexes/base.py @@ -298,7 +298,13 @@ def _memory_usage(self) -> int: def transpose(self) -> Index: return self - def sort_values(self, *, ascending: bool = True, na_position: str = "last"): + def sort_values( + self, + *, + inplace: bool = False, + ascending: bool = True, + na_position: str = "last", + ) -> Index: if na_position not in ["first", "last"]: raise ValueError("Param na_position must be one of 'first' or 'last'") na_last = na_position == "last" diff --git a/bigframes/dataframe.py b/bigframes/dataframe.py index 8e9794a5e3..8ed749138c 100644 --- a/bigframes/dataframe.py +++ b/bigframes/dataframe.py @@ -2234,10 +2234,34 @@ def set_index( col_ids_strs: List[str] = [col_id for col_id in col_ids if col_id is not None] return DataFrame(self._block.set_index(col_ids_strs, append=append, drop=drop)) - @validations.requires_index + @overload # type: ignore[override] def sort_index( - self, ascending: bool = True, na_position: Literal["first", "last"] = "last" + self, + *, + ascending: bool = ..., + inplace: Literal[False] = ..., + na_position: Literal["first", "last"] = ..., ) -> DataFrame: + ... + + @overload + def sort_index( + self, + *, + ascending: bool = ..., + inplace: Literal[True] = ..., + na_position: Literal["first", "last"] = ..., + ) -> None: + ... + + @validations.requires_index + def sort_index( + self, + *, + ascending: bool = True, + inplace: bool = False, + na_position: Literal["first", "last"] = "last", + ) -> Optional[DataFrame]: if na_position not in ["first", "last"]: raise ValueError("Param na_position must be one of 'first' or 'last'") na_last = na_position == "last" @@ -2248,16 +2272,46 @@ def sort_index( else order.descending_over(column, na_last) for column in index_columns ] - return DataFrame(self._block.order_by(ordering)) + block = self._block.order_by(ordering) + if inplace: + self._set_block(block) + return None + else: + return DataFrame(block) + @overload # type: ignore[override] def sort_values( self, by: str | typing.Sequence[str], *, + inplace: Literal[False] = ..., + ascending: bool | typing.Sequence[bool] = ..., + kind: str = ..., + na_position: typing.Literal["first", "last"] = ..., + ) -> DataFrame: + ... + + @overload + def sort_values( + self, + by: str | typing.Sequence[str], + *, + inplace: Literal[True] = ..., + ascending: bool | typing.Sequence[bool] = ..., + kind: str = ..., + na_position: typing.Literal["first", "last"] = ..., + ) -> None: + ... + + def sort_values( + self, + by: str | typing.Sequence[str], + *, + inplace: bool = False, ascending: bool | typing.Sequence[bool] = True, kind: str = "quicksort", na_position: typing.Literal["first", "last"] = "last", - ) -> DataFrame: + ) -> Optional[DataFrame]: if isinstance(by, (bigframes.series.Series, indexes.Index, DataFrame)): raise KeyError( f"Invalid key type: {type(by).__name__}. Please provide valid column name(s)." @@ -2287,7 +2341,12 @@ def sort_values( if is_ascending else order.descending_over(column_id, na_last) ) - return DataFrame(self._block.order_by(ordering)) + block = self._block.order_by(ordering) + if inplace: + self._set_block(block) + return None + else: + return DataFrame(block) def eval(self, expr: str) -> DataFrame: import bigframes.core.eval as bf_eval diff --git a/bigframes/ml/metrics/_metrics.py b/bigframes/ml/metrics/_metrics.py index 658818b261..d7591ef011 100644 --- a/bigframes/ml/metrics/_metrics.py +++ b/bigframes/ml/metrics/_metrics.py @@ -240,7 +240,7 @@ def recall_score( unique_labels = ( bpd.concat([y_true_series, y_pred_series], join="outer") .drop_duplicates() - .sort_values() + .sort_values(inplace=False) ) index = unique_labels.to_list() @@ -277,7 +277,7 @@ def precision_score( unique_labels = ( bpd.concat([y_true_series, y_pred_series], join="outer") .drop_duplicates() - .sort_values() + .sort_values(inplace=False) ) index = unique_labels.to_list() diff --git a/bigframes/series.py b/bigframes/series.py index 37a3723a0a..1e29671310 100644 --- a/bigframes/series.py +++ b/bigframes/series.py @@ -1538,9 +1538,39 @@ def value_counts( ) return Series(block) + @typing.overload # type: ignore[override] def sort_values( - self, *, axis=0, ascending=True, kind: str = "quicksort", na_position="last" + self, + *, + axis=..., + inplace: Literal[True] = ..., + ascending: bool | typing.Sequence[bool] = ..., + kind: str = ..., + na_position: typing.Literal["first", "last"] = ..., + ) -> None: + ... + + @typing.overload + def sort_values( + self, + *, + axis=..., + inplace: Literal[False] = ..., + ascending: bool | typing.Sequence[bool] = ..., + kind: str = ..., + na_position: typing.Literal["first", "last"] = ..., ) -> Series: + ... + + def sort_values( + self, + *, + axis=0, + inplace: bool = False, + ascending=True, + kind: str = "quicksort", + na_position: typing.Literal["first", "last"] = "last", + ) -> Optional[Series]: if axis != 0 and axis != "index": raise ValueError(f"No axis named {axis} for object type Series") if na_position not in ["first", "last"]: @@ -1552,10 +1582,28 @@ def sort_values( else order.descending_over(self._value_column, (na_position == "last")) ], ) - return Series(block) + if inplace: + self._set_block(block) + return None + else: + return Series(block) + + @typing.overload # type: ignore[override] + def sort_index( + self, *, axis=..., inplace: Literal[False] = ..., ascending=..., na_position=... + ) -> Series: + ... + + @typing.overload + def sort_index( + self, *, axis=0, inplace: Literal[True] = ..., ascending=..., na_position=... + ) -> None: + ... @validations.requires_index - def sort_index(self, *, axis=0, ascending=True, na_position="last") -> Series: + def sort_index( + self, *, axis=0, inplace: bool = False, ascending=True, na_position="last" + ) -> Optional[Series]: # TODO(tbergeron): Support level parameter once multi-index introduced. if axis != 0 and axis != "index": raise ValueError(f"No axis named {axis} for object type Series") @@ -1570,7 +1618,11 @@ def sort_index(self, *, axis=0, ascending=True, na_position="last") -> Series: for column in block.index_columns ] block = block.order_by(ordering) - return Series(block) + if inplace: + self._set_block(block) + return None + else: + return Series(block) @validations.requires_ordering() def rolling( diff --git a/tests/system/small/test_dataframe.py b/tests/system/small/test_dataframe.py index 362d736aeb..ce291d4999 100644 --- a/tests/system/small/test_dataframe.py +++ b/tests/system/small/test_dataframe.py @@ -2035,6 +2035,17 @@ def test_sort_index(scalars_dfs, ascending, na_position): pandas.testing.assert_frame_equal(bf_result, pd_result) +def test_dataframe_sort_index_inplace(scalars_dfs): + index_column = "int64_col" + scalars_df, scalars_pandas_df = scalars_dfs + df = scalars_df.copy().set_index(index_column) + df.sort_index(ascending=False, inplace=True) + bf_result = df.to_pandas() + + pd_result = scalars_pandas_df.set_index(index_column).sort_index(ascending=False) + pandas.testing.assert_frame_equal(bf_result, pd_result) + + def test_df_abs(scalars_dfs_maybe_ordered): scalars_df, scalars_pandas_df = scalars_dfs_maybe_ordered columns = ["int64_col", "int64_too", "float64_col"] @@ -2817,6 +2828,32 @@ def test_dataframe_sort_values( ) +@pytest.mark.parametrize( + ("by", "ascending", "na_position"), + [ + ("int64_col", True, "first"), + (["bool_col", "int64_col"], True, "last"), + ], +) +def test_dataframe_sort_values_inplace( + scalars_df_index, scalars_pandas_df_index, by, ascending, na_position +): + # Test needs values to be unique + bf_sorted = scalars_df_index.copy() + bf_sorted.sort_values( + by, ascending=ascending, na_position=na_position, inplace=True + ) + bf_result = bf_sorted.to_pandas() + pd_result = scalars_pandas_df_index.sort_values( + by, ascending=ascending, na_position=na_position + ) + + pandas.testing.assert_frame_equal( + bf_result, + pd_result, + ) + + def test_dataframe_sort_values_invalid_input(scalars_df_index): with pytest.raises(KeyError): scalars_df_index.sort_values(by=scalars_df_index["int64_col"]) diff --git a/tests/system/small/test_series.py b/tests/system/small/test_series.py index c63bf8e12b..3852c417fa 100644 --- a/tests/system/small/test_series.py +++ b/tests/system/small/test_series.py @@ -3339,6 +3339,19 @@ def test_sort_values(scalars_df_index, scalars_pandas_df_index, ascending, na_po ) +def test_series_sort_values_inplace(scalars_df_index, scalars_pandas_df_index): + # Test needs values to be unique + bf_series = scalars_df_index["int64_col"].copy() + bf_series.sort_values(ascending=False, inplace=True) + bf_result = bf_series.to_pandas() + pd_result = scalars_pandas_df_index["int64_col"].sort_values(ascending=False) + + pd.testing.assert_series_equal( + bf_result, + pd_result, + ) + + @pytest.mark.parametrize( ("ascending"), [ @@ -3358,6 +3371,18 @@ def test_sort_index(scalars_df_index, scalars_pandas_df_index, ascending): ) +def test_series_sort_index_inplace(scalars_df_index, scalars_pandas_df_index): + bf_series = scalars_df_index["int64_too"].copy() + bf_series.sort_index(ascending=False, inplace=True) + bf_result = bf_series.to_pandas() + pd_result = scalars_pandas_df_index["int64_too"].sort_index(ascending=False) + + pd.testing.assert_series_equal( + bf_result, + pd_result, + ) + + def test_mask_default_value(scalars_dfs): scalars_df, scalars_pandas_df = scalars_dfs diff --git a/tests/system/small/test_session.py b/tests/system/small/test_session.py index ab460d5bc9..eeb242e8da 100644 --- a/tests/system/small/test_session.py +++ b/tests/system/small/test_session.py @@ -96,16 +96,17 @@ def test_read_gbq_tokyo( tokyo_location: str, ): df = session_tokyo.read_gbq(scalars_table_tokyo, index_col=["rowindex"]) - result = df.sort_index().to_pandas() + df.sort_index(inplace=True) expected = scalars_pandas_df_index # use_explicit_destination=True, otherwise might use path with no query_job - result = session_tokyo._executor.execute( + exec_result = session_tokyo._executor.execute( df._block.expr, use_explicit_destination=True ) - assert result.query_job.location == tokyo_location + assert exec_result.query_job is not None + assert exec_result.query_job.location == tokyo_location - assert len(expected) == result.total_rows + assert len(expected) == exec_result.total_rows @pytest.mark.parametrize( diff --git a/third_party/bigframes_vendored/pandas/core/frame.py b/third_party/bigframes_vendored/pandas/core/frame.py index 8f3e150606..5bbf72b421 100644 --- a/third_party/bigframes_vendored/pandas/core/frame.py +++ b/third_party/bigframes_vendored/pandas/core/frame.py @@ -2213,10 +2213,11 @@ def sort_values( self, by: str | Sequence[str], *, + inplace: bool = False, ascending: bool | Sequence[bool] = True, kind: str = "quicksort", - na_position="last", - ) -> DataFrame: + na_position: Literal["first", "last"] = "last", + ): """Sort by the values along row axis. **Examples:** @@ -2300,6 +2301,8 @@ def sort_values( Sort ascending vs. descending. Specify list for multiple sort orders. If this is a list of bools, must match the length of the by. + inplace (bool, default False): + If True, perform operation in-place. kind (str, default 'quicksort'): Choice of sorting algorithm. Accepts 'quicksort', 'mergesort', 'heapsort', 'stable'. Ignored except when determining whether to @@ -2309,8 +2312,8 @@ def sort_values( if `first`; `last` puts NaNs at the end. Returns: - bigframes.pandas.DataFrame: - DataFrame with sorted values. + bigframes.pandas.DataFram or None: + DataFrame with sorted values or None if inplace=True. Raises: ValueError: @@ -2320,12 +2323,25 @@ def sort_values( def sort_index( self, - ) -> DataFrame: + *, + ascending: bool = True, + inplace: bool = False, + na_position: Literal["first", "last"] = "last", + ): """Sort object by labels (along an axis). + Args: + ascending (bool, default True) + Sort ascending vs. descending. + inplace (bool, default False): + Whether to modify the DataFrame rather than creating a new one. + na_position ({'first', 'last'}, default 'last'): + Puts NaNs at the beginning if `first`; `last` puts NaNs at the end. + Not implemented for MultiIndex. + Returns: bigframes.pandas.DataFrame: - The original DataFrame sorted by the labels. + DataFrame with sorted values or None if inplace=True. Raises: ValueError: diff --git a/third_party/bigframes_vendored/pandas/core/series.py b/third_party/bigframes_vendored/pandas/core/series.py index a2d0983652..8164fa7415 100644 --- a/third_party/bigframes_vendored/pandas/core/series.py +++ b/third_party/bigframes_vendored/pandas/core/series.py @@ -1504,10 +1504,11 @@ def sort_values( self, *, axis: Axis = 0, + inplace: bool = False, ascending: bool | int | Sequence[bool] | Sequence[int] = True, kind: str = "quicksort", na_position: str = "last", - ) -> Series | None: + ): """ Sort by the values. @@ -1581,6 +1582,8 @@ def sort_values( Args: axis (0 or 'index'): Unused. Parameter needed for compatibility with DataFrame. + inplace (bool, default False): + Whether to modify the Series rather than creating a new one. ascending (bool or list of bools, default True): If True, sort values in ascending order, otherwise descending. kind (str, default to 'quicksort'): @@ -1601,9 +1604,10 @@ def sort_index( self, *, axis: Axis = 0, + inplace: bool = False, ascending: bool | Sequence[bool] = True, na_position: NaPosition = "last", - ) -> Series | None: + ): """ Sort Series by index labels. @@ -1647,6 +1651,8 @@ def sort_index( Args: axis ({0 or 'index'}): Unused. Parameter needed for compatibility with DataFrame. + inplace (bool, default False): + Whether to modify the Series rather than creating a new one. ascending (bool or list-like of bools, default True): Sort ascending vs. descending. When the index is a MultiIndex the sort direction can be controlled for each level individually. From 6b82781b9bb78fc97d735fd3f8c999cf580bd7e2 Mon Sep 17 00:00:00 2001 From: Garrett Wu <6505921+GarrettWu@users.noreply.github.com> Date: Thu, 8 May 2025 15:28:04 -0700 Subject: [PATCH 22/36] chore: add exif test image in scripts (#1711) --- scripts/data/images_exif/test_image_exif.jpg | Bin 0 -> 1396 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 scripts/data/images_exif/test_image_exif.jpg diff --git a/scripts/data/images_exif/test_image_exif.jpg b/scripts/data/images_exif/test_image_exif.jpg new file mode 100644 index 0000000000000000000000000000000000000000..fdfdaf9ad080b7264236111519a1ae3b597484bf GIT binary patch literal 1396 zcmex=Mp4OGdX55x?RY=j$ zkxe)-kzJ`!#HexNLJno8jR!@8E`CrkPAY2R|V^&07y2J$~}^+4C1KUw!=a`ODXD z-+%o41@ado12e>1KoYCJ1cClyVqsxsVF&q(k*OTyaX}VVMME|r$3XVPLSZGNMh+3D zi3>L#L=tVK7M30Q3Kw0Mr7$B>(^b literal 0 HcmV?d00001 From 68b6532a780d6349a4b65994b696c8026457eb94 Mon Sep 17 00:00:00 2001 From: Garrett Wu <6505921+GarrettWu@users.noreply.github.com> Date: Thu, 8 May 2025 15:33:28 -0700 Subject: [PATCH 23/36] docs: move multimodal notebook out of experimental folder (#1712) --- .../experimental/multimodal_dataframe.ipynb | 1073 ----------------- .../multimodal/multimodal_dataframe.ipynb | 449 +++++++ noxfile.py | 2 +- 3 files changed, 450 insertions(+), 1074 deletions(-) delete mode 100644 notebooks/experimental/multimodal_dataframe.ipynb create mode 100644 notebooks/multimodal/multimodal_dataframe.ipynb diff --git a/notebooks/experimental/multimodal_dataframe.ipynb b/notebooks/experimental/multimodal_dataframe.ipynb deleted file mode 100644 index 05b133a345..0000000000 --- a/notebooks/experimental/multimodal_dataframe.ipynb +++ /dev/null @@ -1,1073 +0,0 @@ -{ - "cells": [ - { - "cell_type": "code", - "execution_count": 1, - "metadata": {}, - "outputs": [], - "source": [ - "# Copyright 2025 Google LLC\n", - "#\n", - "# Licensed under the Apache License, Version 2.0 (the \"License\");\n", - "# you may not use this file except in compliance with the License.\n", - "# You may obtain a copy of the License at\n", - "#\n", - "# https://www.apache.org/licenses/LICENSE-2.0\n", - "#\n", - "# Unless required by applicable law or agreed to in writing, software\n", - "# distributed under the License is distributed on an \"AS IS\" BASIS,\n", - "# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n", - "# See the License for the specific language governing permissions and\n", - "# limitations under the License." - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "YOrUAvz6DMw-" - }, - "source": [ - "# BigFrames Multimodal DataFrame\n", - "\n", - "\n", - "\n", - " \n", - " \n", - " \n", - "
\n", - " \n", - " \"Colab Run in Colab\n", - " \n", - " \n", - " \n", - " \"GitHub\n", - " View on GitHub\n", - " \n", - " \n", - " \n", - " \"BQ\n", - " Open in BQ Studio\n", - " \n", - "
\n" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "This notebook is introducing BigFrames experimental Multimodal features:\n", - "1. Create Multimodal DataFrame\n", - "2. Combine unstructured data with structured data\n", - "3. Conduct image transformations\n", - "4. Use LLM models to ask questions and generate embeddings on images\n", - "5. PDF chunking function" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "PEAJQQ6AFg-n" - }, - "source": [ - "### Setup" - ] - }, - { - "cell_type": "code", - "execution_count": 2, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/" - }, - "id": "bGyhLnfEeB0X", - "outputId": "83ac8b64-3f44-4d43-d089-28a5026cbb42" - }, - "outputs": [], - "source": [ - "PROJECT = \"bigframes-dev\" # replace with your project, project needs to be allowlisted go/bq-multimodal-allowlist (internal)\n", - "# User must have https://cloud.google.com/bigquery/docs/use-bigquery-dataframes#permissions to use bigframes, BQ connection admin/user to create/use connections, BQ ObjRef permissions for ObjectRef and BQ routines permissions for using transform functions.\n", - "# Or simply has BQ Admin role for all.\n", - "\n", - "import bigframes\n", - "# Setup project\n", - "bigframes.options.bigquery.project = PROJECT\n", - "\n", - "# Display options\n", - "bigframes.options.display.blob_display_width = 300\n", - "bigframes.options.display.progress_bar = None\n", - "\n", - "import bigframes.pandas as bpd" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "ifKOq7VZGtZy" - }, - "source": [ - "### 1. Create Multimodal DataFrame\n", - "There are several ways to create Multimodal DataFrame. The easiest way is from the wildcard paths." - ] - }, - { - "cell_type": "code", - "execution_count": 3, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/" - }, - "id": "fx6YcZJbeYru", - "outputId": "d707954a-0dd0-4c50-b7bf-36b140cf76cf" - }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/global_session.py:114: DefaultLocationWarning: No explicit location is set, so using location US for the session.\n", - " return func(get_global_session(), *args, **kwargs)\n" - ] - } - ], - "source": [ - "# Create blob columns from wildcard path.\n", - "df_image = bpd.from_glob_path(\n", - " \"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/*\", name=\"image\"\n", - ")\n", - "# Other ways are: from string uri column\n", - "# df = bpd.DataFrame({\"uri\": [\"gs:///\", \"gs:///\"]})\n", - "# df[\"blob_col\"] = df[\"uri\"].str.to_blob()\n", - "\n", - "# From an existing object table\n", - "# df = bpd.read_gbq_object_table(\"\", name=\"blob_col\")" - ] - }, - { - "cell_type": "code", - "execution_count": 4, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/", - "height": 487 - }, - "id": "HhCb8jRsLe9B", - "outputId": "03081cf9-3a22-42c9-b38f-649f592fdada" - }, - "outputs": [ - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
image
0
1
2
3
4
\n", - "

5 rows × 1 columns

\n", - "
[5 rows x 1 columns in total]" - ], - "text/plain": [ - " image\n", - "0 {'uri': 'gs://cloud-samples-data/bigquery/tuto...\n", - "1 {'uri': 'gs://cloud-samples-data/bigquery/tuto...\n", - "2 {'uri': 'gs://cloud-samples-data/bigquery/tuto...\n", - "3 {'uri': 'gs://cloud-samples-data/bigquery/tuto...\n", - "4 {'uri': 'gs://cloud-samples-data/bigquery/tuto...\n", - "\n", - "[5 rows x 1 columns]" - ] - }, - "execution_count": 4, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "# Take only the 5 images to deal with. Preview the content of the Mutimodal DataFrame\n", - "df_image = df_image.head(5)\n", - "df_image" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "b6RRZb3qPi_T" - }, - "source": [ - "### 2. Combine unstructured data with structured data" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "4YJCdmLtR-qu" - }, - "source": [ - "Now you can put more information into the table to describe the files. Such as author info from inputs, or other metadata from the gcs object itself." - ] - }, - { - "cell_type": "code", - "execution_count": 5, - "metadata": { - "id": "YYYVn7NDH0Me" - }, - "outputs": [ - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
imageauthorcontent_typesizeupdated
0aliceimage/png15912402025-03-20 17:45:04+00:00
1bobimage/png11829512025-03-20 17:45:02+00:00
2bobimage/png15208842025-03-20 17:44:55+00:00
3aliceimage/png12354012025-03-20 17:45:19+00:00
4bobimage/png15919232025-03-20 17:44:47+00:00
\n", - "

5 rows × 5 columns

\n", - "
[5 rows x 5 columns in total]" - ], - "text/plain": [ - " image author content_type \\\n", - "0 {'uri': 'gs://cloud-samples-data/bigquery/tuto... alice image/png \n", - "1 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", - "2 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", - "3 {'uri': 'gs://cloud-samples-data/bigquery/tuto... alice image/png \n", - "4 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", - "\n", - " size updated \n", - "0 1591240 2025-03-20 17:45:04+00:00 \n", - "1 1182951 2025-03-20 17:45:02+00:00 \n", - "2 1520884 2025-03-20 17:44:55+00:00 \n", - "3 1235401 2025-03-20 17:45:19+00:00 \n", - "4 1591923 2025-03-20 17:44:47+00:00 \n", - "\n", - "[5 rows x 5 columns]" - ] - }, - "execution_count": 5, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "# Combine unstructured data with structured data\n", - "df_image[\"author\"] = [\"alice\", \"bob\", \"bob\", \"alice\", \"bob\"] # type: ignore\n", - "df_image[\"content_type\"] = df_image[\"image\"].blob.content_type()\n", - "df_image[\"size\"] = df_image[\"image\"].blob.size()\n", - "df_image[\"updated\"] = df_image[\"image\"].blob.updated()\n", - "df_image" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "NUd4Kog_QLRS" - }, - "source": [ - "Then you can filter the rows based on the structured data. And for different content types, you can display them respectively or together." - ] - }, - { - "cell_type": "code", - "execution_count": 6, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/", - "height": 75 - }, - "id": "UGuAk9PNDRF3", - "outputId": "73feb33d-4a05-48fb-96e5-3c48c2a456f3" - }, - "outputs": [ - { - "data": { - "text/html": [ - "" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "# filter images and display, you can also display audio and video types\n", - "df_image[df_image[\"author\"] == \"alice\"][\"image\"].blob.display()" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "1IJuakwJTZey" - }, - "source": [ - "### 3. Conduct image transformations\n", - "BigFrames Multimodal DataFrame provides image(and other) transformation functions. Such as image_blur, image_resize and image_normalize. The output can be saved to GCS folders or to BQ as bytes." - ] - }, - { - "cell_type": "code", - "execution_count": 7, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/" - }, - "id": "VWsl5BBPJ6N7", - "outputId": "45d2356e-322b-4982-cfa7-42d034dc4344" - }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/log_adapter.py:164: FunctionAxisOnePreviewWarning: Blob Functions use bigframes DataFrame Managed function with axis=1 senario, which is a preview feature.\n", - " return method(self, *args, **kwargs)\n", - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/log_adapter.py:164: FunctionAxisOnePreviewWarning: Blob Functions use bigframes DataFrame Managed function with axis=1 senario, which is a preview feature.\n", - " return method(self, *args, **kwargs)\n", - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/log_adapter.py:164: FunctionAxisOnePreviewWarning: Blob Functions use bigframes DataFrame Managed function with axis=1 senario, which is a preview feature.\n", - " return method(self, *args, **kwargs)\n" - ] - } - ], - "source": [ - "df_image[\"blurred\"] = df_image[\"image\"].blob.image_blur(\n", - " (20, 20), dst=\"gs://bigframes_blob_test/image_blur_transformed/\"\n", - ")\n", - "df_image[\"resized\"] = df_image[\"image\"].blob.image_resize(\n", - " (300, 200), dst=\"gs://bigframes_blob_test/image_resize_transformed/\"\n", - ")\n", - "df_image[\"normalized\"] = df_image[\"image\"].blob.image_normalize(\n", - " alpha=50.0,\n", - " beta=150.0,\n", - " norm_type=\"minmax\",\n", - " dst=\"gs://bigframes_blob_test/image_normalize_transformed/\",\n", - ")" - ] - }, - { - "cell_type": "code", - "execution_count": 8, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/" - }, - "id": "rWCAGC8w64vU", - "outputId": "d7d456f0-8b56-492c-fe1b-967e9664d813" - }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/log_adapter.py:164: FunctionAxisOnePreviewWarning: Blob Functions use bigframes DataFrame Managed function with axis=1 senario, which is a preview feature.\n", - " return method(self, *args, **kwargs)\n" - ] - } - ], - "source": [ - "# You can also chain functions together\n", - "df_image[\"blur_resized\"] = df_image[\"blurred\"].blob.image_resize((300, 200), dst=\"gs://bigframes_blob_test/image_blur_resize_transformed/\")" - ] - }, - { - "cell_type": "code", - "execution_count": 9, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/", - "height": 605 - }, - "id": "6NGK6GYSU44B", - "outputId": "859101c1-2ee4-4f9a-e250-e8947127420a" - }, - "outputs": [ - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
imageauthorcontent_typesizeupdatedblurredresizednormalizedblur_resized
0aliceimage/png15912402025-03-20 17:45:04+00:00
1bobimage/png11829512025-03-20 17:45:02+00:00
2bobimage/png15208842025-03-20 17:44:55+00:00
3aliceimage/png12354012025-03-20 17:45:19+00:00
4bobimage/png15919232025-03-20 17:44:47+00:00
\n", - "

5 rows × 9 columns

\n", - "
[5 rows x 9 columns in total]" - ], - "text/plain": [ - " image author content_type \\\n", - "0 {'uri': 'gs://cloud-samples-data/bigquery/tuto... alice image/png \n", - "1 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", - "2 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", - "3 {'uri': 'gs://cloud-samples-data/bigquery/tuto... alice image/png \n", - "4 {'uri': 'gs://cloud-samples-data/bigquery/tuto... bob image/png \n", - "\n", - " size updated \\\n", - "0 1591240 2025-03-20 17:45:04+00:00 \n", - "1 1182951 2025-03-20 17:45:02+00:00 \n", - "2 1520884 2025-03-20 17:44:55+00:00 \n", - "3 1235401 2025-03-20 17:45:19+00:00 \n", - "4 1591923 2025-03-20 17:44:47+00:00 \n", - "\n", - " blurred \\\n", - "0 {'uri': 'gs://bigframes_blob_test/image_blur_t... \n", - "1 {'uri': 'gs://bigframes_blob_test/image_blur_t... \n", - "2 {'uri': 'gs://bigframes_blob_test/image_blur_t... \n", - "3 {'uri': 'gs://bigframes_blob_test/image_blur_t... \n", - "4 {'uri': 'gs://bigframes_blob_test/image_blur_t... \n", - "\n", - " resized \\\n", - "0 {'uri': 'gs://bigframes_blob_test/image_resize... \n", - "1 {'uri': 'gs://bigframes_blob_test/image_resize... \n", - "2 {'uri': 'gs://bigframes_blob_test/image_resize... \n", - "3 {'uri': 'gs://bigframes_blob_test/image_resize... \n", - "4 {'uri': 'gs://bigframes_blob_test/image_resize... \n", - "\n", - " normalized \\\n", - "0 {'uri': 'gs://bigframes_blob_test/image_normal... \n", - "1 {'uri': 'gs://bigframes_blob_test/image_normal... \n", - "2 {'uri': 'gs://bigframes_blob_test/image_normal... \n", - "3 {'uri': 'gs://bigframes_blob_test/image_normal... \n", - "4 {'uri': 'gs://bigframes_blob_test/image_normal... \n", - "\n", - " blur_resized \n", - "0 {'uri': 'gs://bigframes_blob_test/image_blur_r... \n", - "1 {'uri': 'gs://bigframes_blob_test/image_blur_r... \n", - "2 {'uri': 'gs://bigframes_blob_test/image_blur_r... \n", - "3 {'uri': 'gs://bigframes_blob_test/image_blur_r... \n", - "4 {'uri': 'gs://bigframes_blob_test/image_blur_r... \n", - "\n", - "[5 rows x 9 columns]" - ] - }, - "execution_count": 9, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "df_image" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "Euk5saeVVdTP" - }, - "source": [ - "### 4. Use LLM models to ask questions and generate embeddings on images" - ] - }, - { - "cell_type": "code", - "execution_count": 10, - "metadata": { - "id": "mRUGfcaFVW-3" - }, - "outputs": [], - "source": [ - "from bigframes.ml import llm\n", - "gemini = llm.GeminiTextGenerator(model_name=\"gemini-1.5-flash-002\")" - ] - }, - { - "cell_type": "code", - "execution_count": 11, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/", - "height": 657 - }, - "id": "DNFP7CbjWdR9", - "outputId": "3f90a062-0abc-4bce-f53c-db57b06a14b9" - }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:109: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", - "`db_dtypes` is a preview feature and subject to change.\n", - " warnings.warn(msg, bfe.PreviewWarning)\n" - ] - }, - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
ml_generate_text_llm_resultimage
0That's a tin of **K9Guard Dog Paw Balm**.
1That's a bottle of **K9 Guard Dog Hot Spot Spray**. It's a pet product designed to soothe and protect dogs' hot spots (irritated areas of skin).
\n", - "

2 rows × 2 columns

\n", - "
[2 rows x 2 columns in total]" - ], - "text/plain": [ - " ml_generate_text_llm_result \\\n", - "0 That's a tin of **K9Guard Dog Paw Balm**.\n", - " \n", - "1 That's a bottle of **K9 Guard Dog Hot Spot Spr... \n", - "\n", - " image \n", - "0 {'uri': 'gs://cloud-samples-data/bigquery/tuto... \n", - "1 {'uri': 'gs://cloud-samples-data/bigquery/tuto... \n", - "\n", - "[2 rows x 2 columns]" - ] - }, - "execution_count": 11, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "# Ask the same question on the images\n", - "df_image = df_image.head(2)\n", - "answer = gemini.predict(df_image, prompt=[\"what item is it?\", df_image[\"image\"]])\n", - "answer[[\"ml_generate_text_llm_result\", \"image\"]]" - ] - }, - { - "cell_type": "code", - "execution_count": 12, - "metadata": { - "id": "IG3J3HsKhyBY" - }, - "outputs": [], - "source": [ - "# Ask different questions\n", - "df_image[\"question\"] = [\"what item is it?\", \"what color is the picture?\"]" - ] - }, - { - "cell_type": "code", - "execution_count": 13, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/", - "height": 657 - }, - "id": "qKOb765IiVuD", - "outputId": "731bafad-ea29-463f-c8c1-cb7acfd70e5d" - }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:109: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", - "`db_dtypes` is a preview feature and subject to change.\n", - " warnings.warn(msg, bfe.PreviewWarning)\n" - ] - }, - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
ml_generate_text_llm_resultimage
0That's a tin of **K9Guard Dog Paw Balm**. It's a balm designed to protect and care for a dog's paws.
1The background of the picture is light gray. The bottle is predominantly white with teal/light blue accents on the spray nozzle and text. There is a small image of an aloe vera plant on the label.
\n", - "

2 rows × 2 columns

\n", - "
[2 rows x 2 columns in total]" - ], - "text/plain": [ - " ml_generate_text_llm_result \\\n", - "0 That's a tin of **K9Guard Dog Paw Balm**. It'... \n", - "1 The background of the picture is light gray. ... \n", - "\n", - " image \n", - "0 {'uri': 'gs://cloud-samples-data/bigquery/tuto... \n", - "1 {'uri': 'gs://cloud-samples-data/bigquery/tuto... \n", - "\n", - "[2 rows x 2 columns]" - ] - }, - "execution_count": 13, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "answer_alt = gemini.predict(df_image, prompt=[df_image[\"question\"], df_image[\"image\"]])\n", - "answer_alt[[\"ml_generate_text_llm_result\", \"image\"]]" - ] - }, - { - "cell_type": "code", - "execution_count": 14, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/", - "height": 300 - }, - "id": "KATVv2CO5RT1", - "outputId": "6ec01f27-70b6-4f69-c545-e5e3c879480c" - }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/log_adapter.py:164: FutureWarning: Since upgrading the default model can cause unintended breakages, the\n", - "default model will be removed in BigFrames 3.0. Please supply an\n", - "explicit model to avoid this message.\n", - " return method(self, *args, **kwargs)\n", - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/array_value.py:109: PreviewWarning: JSON column interpretation as a custom PyArrow extention in\n", - "`db_dtypes` is a preview feature and subject to change.\n", - " warnings.warn(msg, bfe.PreviewWarning)\n" - ] - }, - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
ml_generate_embedding_resultml_generate_embedding_statusml_generate_embedding_start_secml_generate_embedding_end_seccontent
0[ 0.00638822 0.01666385 0.00451817 ... -0.02684802 -0.00475593\n", - " -0.01989058]{\"access_urls\":{\"expiry_time\":\"2025-05-06T06:14:15Z\",\"read_url\":\"https://storage.googleapis.com/cloud-samples-data/bigquery%2Ftutorials%2Fcymbal-pets%2Fimages%2Fk9-guard-dog-paw-balm.png?X-Goog-Algorithm=GOOG4-RSA-SHA256&X-Goog-Credential=bqcx-1084210331973-pcbl%40gcp-sa-bigquery-condel.iam.gserviceaccount.com%2F20250506%2Fauto%2Fstorage%2Fgoog4_request&X-Goog-Date=20250506T001415Z&X-Goog-Expires=21600&X-Goog-SignedHeaders=host&generation=1742492703986347&X-Goog-Signature=4ff6735f3e2d1b9aede89b881884c8413f7b5fde1c9c65b140de7eed1bdaca3609cd77e5dfb81e83fbc77adf20dbbec687ab0dd3f9fadd7ffd9961b1eecf14464d8257ab71dc509859eda505877a16a6b97a34bdce2a36b93da14ffb9e4de5b6c60b8c49e249efa56cfa144c5855d92fb67ee8a09f753d9889ff69c8bef70f01224bcfdda66341bd8cd7d56cc4fdc2a5618d1cebc654077890afa5329c05bb8accf440802ad3aeaa624f23725f7e9bec9270d4379c30f7dd4c4ca99752103396f87872e9484fe3efcd6514754d85d825d56bed140e49a15985bee0274fc6d4f5eceecc762144d451454896b5fcda4cade198c2b7bf07e5186556a393b3b85264\",\"write_url\":\"\"},\"objectref\":{\"authorizer\":\"bigframes-dev.us.bigframes-default-connection\",\"details\":{\"gcs_metadata\":{\"content_type\":\"image/png\",\"md5_hash\":\"c46ac29292d8ba244101dcada3ea86d5\",\"size\":1591240,\"updated\":1742492704000000}},\"uri\":\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/k9-guard-dog-paw-balm.png\",\"version\":\"1742492703986347\"}}
1[ 0.00973672 0.02148364 0.00244308 ... 0.00462242 0.0131027\n", - " -0.00038765]{\"access_urls\":{\"expiry_time\":\"2025-05-06T06:14:15Z\",\"read_url\":\"https://storage.googleapis.com/cloud-samples-data/bigquery%2Ftutorials%2Fcymbal-pets%2Fimages%2Fk9-guard-dog-hot-spot-spray.png?X-Goog-Algorithm=GOOG4-RSA-SHA256&X-Goog-Credential=bqcx-1084210331973-pcbl%40gcp-sa-bigquery-condel.iam.gserviceaccount.com%2F20250506%2Fauto%2Fstorage%2Fgoog4_request&X-Goog-Date=20250506T001415Z&X-Goog-Expires=21600&X-Goog-SignedHeaders=host&generation=1742492702954473&X-Goog-Signature=4adc1cf96fe21f385782ea26d52faf9093995b334f592ab0b3a791dc47402b5626c82b9c31c3ecd553d6ccf16012d42c0f7dd8e07b641c60515e43e33bc64da60082763f1567bc9ff2f37a19d3e33cf2a87a9872512b9400265b42092c8070254d842a26cc28d0332b86d0bc052a71ea4bd85d026604f81235d9ee367852e912ad5b5ea405023f92c269586a8fa417b1b7ffe8026086f1d9aaf893635334715568025d0f1d7a2108b33cde9a9012e0684763a6b1743e8decbf7bc9e9f582c7f72bcc2bf31254253ef3c21cb2d10c4c1580e98729eb0611edaea2690bf4b4884449a44851fb0f788a7692dc94bf4fe3c2e287566b6b92e4cc880870adfc4093c4\",\"write_url\":\"\"},\"objectref\":{\"authorizer\":\"bigframes-dev.us.bigframes-default-connection\",\"details\":{\"gcs_metadata\":{\"content_type\":\"image/png\",\"md5_hash\":\"ae802a64dfaeaf556609429fcbe02542\",\"size\":1182951,\"updated\":1742492702000000}},\"uri\":\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/k9-guard-dog-hot-spot-spray.png\",\"version\":\"1742492702954473\"}}
\n", - "

2 rows × 5 columns

\n", - "
[2 rows x 5 columns in total]" - ], - "text/plain": [ - " ml_generate_embedding_result \\\n", - "0 [ 0.00638822 0.01666385 0.00451817 ... -0.02... \n", - "1 [ 0.00973672 0.02148364 0.00244308 ... 0.00... \n", - "\n", - " ml_generate_embedding_status ml_generate_embedding_start_sec \\\n", - "0 \n", - "1 \n", - "\n", - " ml_generate_embedding_end_sec \\\n", - "0 \n", - "1 \n", - "\n", - " content \n", - "0 {\"access_urls\":{\"expiry_time\":\"2025-05-06T06:1... \n", - "1 {\"access_urls\":{\"expiry_time\":\"2025-05-06T06:1... \n", - "\n", - "[2 rows x 5 columns]" - ] - }, - "execution_count": 14, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "# Generate embeddings.\n", - "embed_model = llm.MultimodalEmbeddingGenerator()\n", - "embeddings = embed_model.predict(df_image[\"image\"])\n", - "embeddings" - ] - }, - { - "cell_type": "markdown", - "metadata": { - "id": "iRUi8AjG7cIf" - }, - "source": [ - "### 5. PDF chunking function" - ] - }, - { - "cell_type": "code", - "execution_count": 15, - "metadata": { - "id": "oDDuYtUm5Yiy" - }, - "outputs": [], - "source": [ - "df_pdf = bpd.from_glob_path(\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/documents/*\", name=\"pdf\")" - ] - }, - { - "cell_type": "code", - "execution_count": 16, - "metadata": { - "colab": { - "base_uri": "https://localhost:8080/" - }, - "id": "7jLpMYaj7nj8", - "outputId": "06d5456f-580f-4693-adff-2605104b056c" - }, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/usr/local/google/home/garrettwu/src/bigframes/bigframes/core/log_adapter.py:164: FunctionAxisOnePreviewWarning: Blob Functions use bigframes DataFrame Managed function with axis=1 senario, which is a preview feature.\n", - " return method(self, *args, **kwargs)\n" - ] - } - ], - "source": [ - "df_pdf[\"chunked\"] = df_pdf[\"pdf\"].blob.pdf_chunk()" - ] - }, - { - "cell_type": "code", - "execution_count": 17, - "metadata": { - "id": "kaPvJATN7zlw" - }, - "outputs": [ - { - "data": { - "text/plain": [ - "0 CritterCuisine Pro 5000 - Automatic Pet Feeder...\n", - "0 on a level, stable surface to prevent tipping....\n", - "0 included)\n", - "to maintain the schedule during powe...\n", - "0 digits for Meal 1 will flash.\n", - "\u0000. Use the UP/DO...\n", - "0 paperclip) for 5\n", - "seconds. This will reset all ...\n", - "0 unit with a damp cloth. Do not immerse the bas...\n", - "0 continues,\n", - "contact customer support.\n", - "E2: Food ...\n", - "Name: chunked, dtype: string" - ] - }, - "execution_count": 17, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "chunked = df_pdf[\"chunked\"].explode()\n", - "chunked" - ] - }, - { - "cell_type": "code", - "execution_count": null, - "metadata": {}, - "outputs": [], - "source": [] - } - ], - "metadata": { - "colab": { - "provenance": [] - }, - "kernelspec": { - "display_name": "Python 3", - "name": "python3" - }, - "language_info": { - "codemirror_mode": { - "name": "ipython", - "version": 3 - }, - "file_extension": ".py", - "mimetype": "text/x-python", - "name": "python", - "nbconvert_exporter": "python", - "pygments_lexer": "ipython3", - "version": "3.10.14" - } - }, - "nbformat": 4, - "nbformat_minor": 0 -} diff --git a/notebooks/multimodal/multimodal_dataframe.ipynb b/notebooks/multimodal/multimodal_dataframe.ipynb new file mode 100644 index 0000000000..860d8080eb --- /dev/null +++ b/notebooks/multimodal/multimodal_dataframe.ipynb @@ -0,0 +1,449 @@ +{ + "cells": [ + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Copyright 2025 Google LLC\n", + "#\n", + "# Licensed under the Apache License, Version 2.0 (the \"License\");\n", + "# you may not use this file except in compliance with the License.\n", + "# You may obtain a copy of the License at\n", + "#\n", + "# https://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing, software\n", + "# distributed under the License is distributed on an \"AS IS\" BASIS,\n", + "# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n", + "# See the License for the specific language governing permissions and\n", + "# limitations under the License." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "YOrUAvz6DMw-" + }, + "source": [ + "# BigFrames Multimodal DataFrame\n", + "\n", + "\n", + "\n", + " \n", + " \n", + " \n", + "
\n", + " \n", + " \"Colab Run in Colab\n", + " \n", + " \n", + " \n", + " \"GitHub\n", + " View on GitHub\n", + " \n", + " \n", + " \n", + " \"BQ\n", + " Open in BQ Studio\n", + " \n", + "
\n" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "This notebook is introducing BigFrames experimental Multimodal features:\n", + "1. Create Multimodal DataFrame\n", + "2. Combine unstructured data with structured data\n", + "3. Conduct image transformations\n", + "4. Use LLM models to ask questions and generate embeddings on images\n", + "5. PDF chunking function" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "PEAJQQ6AFg-n" + }, + "source": [ + "### Setup" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "bGyhLnfEeB0X", + "outputId": "83ac8b64-3f44-4d43-d089-28a5026cbb42" + }, + "outputs": [], + "source": [ + "PROJECT = \"bigframes-dev\" # replace with your project, project needs to be allowlisted go/bq-multimodal-allowlist (internal)\n", + "# User must have https://cloud.google.com/bigquery/docs/use-bigquery-dataframes#permissions to use bigframes, BQ connection admin/user to create/use connections, BQ ObjRef permissions for ObjectRef and BQ routines permissions for using transform functions.\n", + "# Or simply has BQ Admin role for all.\n", + "OUTPUT_BUCKET = \"bigframes_blob_test\" # replace with your GCS bucket. The connection (or bigframes-default-connection of the project) must have read/write permission to the bucket.\n", + "\n", + "import bigframes\n", + "# Setup project\n", + "bigframes.options.bigquery.project = PROJECT\n", + "\n", + "# Display options\n", + "bigframes.options.display.blob_display_width = 300\n", + "bigframes.options.display.progress_bar = None\n", + "\n", + "import bigframes.pandas as bpd" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "ifKOq7VZGtZy" + }, + "source": [ + "### 1. Create Multimodal DataFrame\n", + "There are several ways to create Multimodal DataFrame. The easiest way is from the wildcard paths." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "fx6YcZJbeYru", + "outputId": "d707954a-0dd0-4c50-b7bf-36b140cf76cf" + }, + "outputs": [], + "source": [ + "# Create blob columns from wildcard path.\n", + "df_image = bpd.from_glob_path(\n", + " \"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/images/*\", name=\"image\"\n", + ")\n", + "# Other ways are: from string uri column\n", + "# df = bpd.DataFrame({\"uri\": [\"gs:///\", \"gs:///\"]})\n", + "# df[\"blob_col\"] = df[\"uri\"].str.to_blob()\n", + "\n", + "# From an existing object table\n", + "# df = bpd.read_gbq_object_table(\"\", name=\"blob_col\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 487 + }, + "id": "HhCb8jRsLe9B", + "outputId": "03081cf9-3a22-42c9-b38f-649f592fdada" + }, + "outputs": [], + "source": [ + "# Take only the 5 images to deal with. Preview the content of the Mutimodal DataFrame\n", + "df_image = df_image.head(5)\n", + "df_image" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "b6RRZb3qPi_T" + }, + "source": [ + "### 2. Combine unstructured data with structured data" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "4YJCdmLtR-qu" + }, + "source": [ + "Now you can put more information into the table to describe the files. Such as author info from inputs, or other metadata from the gcs object itself." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "YYYVn7NDH0Me" + }, + "outputs": [], + "source": [ + "# Combine unstructured data with structured data\n", + "df_image[\"author\"] = [\"alice\", \"bob\", \"bob\", \"alice\", \"bob\"] # type: ignore\n", + "df_image[\"content_type\"] = df_image[\"image\"].blob.content_type()\n", + "df_image[\"size\"] = df_image[\"image\"].blob.size()\n", + "df_image[\"updated\"] = df_image[\"image\"].blob.updated()\n", + "df_image" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "NUd4Kog_QLRS" + }, + "source": [ + "Then you can filter the rows based on the structured data. And for different content types, you can display them respectively or together." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 75 + }, + "id": "UGuAk9PNDRF3", + "outputId": "73feb33d-4a05-48fb-96e5-3c48c2a456f3" + }, + "outputs": [], + "source": [ + "# filter images and display, you can also display audio and video types\n", + "df_image[df_image[\"author\"] == \"alice\"][\"image\"].blob.display()" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "1IJuakwJTZey" + }, + "source": [ + "### 3. Conduct image transformations\n", + "BigFrames Multimodal DataFrame provides image(and other) transformation functions. Such as image_blur, image_resize and image_normalize. The output can be saved to GCS folders or to BQ as bytes." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "VWsl5BBPJ6N7", + "outputId": "45d2356e-322b-4982-cfa7-42d034dc4344" + }, + "outputs": [], + "source": [ + "df_image[\"blurred\"] = df_image[\"image\"].blob.image_blur(\n", + " (20, 20), dst=f\"gs://{OUTPUT_BUCKET}/image_blur_transformed/\"\n", + ")\n", + "df_image[\"resized\"] = df_image[\"image\"].blob.image_resize(\n", + " (300, 200), dst=f\"gs://{OUTPUT_BUCKET}/image_resize_transformed/\"\n", + ")\n", + "df_image[\"normalized\"] = df_image[\"image\"].blob.image_normalize(\n", + " alpha=50.0,\n", + " beta=150.0,\n", + " norm_type=\"minmax\",\n", + " dst=f\"gs://{OUTPUT_BUCKET}/image_normalize_transformed/\",\n", + ")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "rWCAGC8w64vU", + "outputId": "d7d456f0-8b56-492c-fe1b-967e9664d813" + }, + "outputs": [], + "source": [ + "# You can also chain functions together\n", + "df_image[\"blur_resized\"] = df_image[\"blurred\"].blob.image_resize((300, 200), dst=f\"gs://{OUTPUT_BUCKET}/image_blur_resize_transformed/\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 605 + }, + "id": "6NGK6GYSU44B", + "outputId": "859101c1-2ee4-4f9a-e250-e8947127420a" + }, + "outputs": [], + "source": [ + "df_image" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "Euk5saeVVdTP" + }, + "source": [ + "### 4. Use LLM models to ask questions and generate embeddings on images" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "mRUGfcaFVW-3" + }, + "outputs": [], + "source": [ + "from bigframes.ml import llm\n", + "gemini = llm.GeminiTextGenerator()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 657 + }, + "id": "DNFP7CbjWdR9", + "outputId": "3f90a062-0abc-4bce-f53c-db57b06a14b9" + }, + "outputs": [], + "source": [ + "# Ask the same question on the images\n", + "df_image = df_image.head(2)\n", + "answer = gemini.predict(df_image, prompt=[\"what item is it?\", df_image[\"image\"]])\n", + "answer[[\"ml_generate_text_llm_result\", \"image\"]]" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "IG3J3HsKhyBY" + }, + "outputs": [], + "source": [ + "# Ask different questions\n", + "df_image[\"question\"] = [\"what item is it?\", \"what color is the picture?\"]" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 657 + }, + "id": "qKOb765IiVuD", + "outputId": "731bafad-ea29-463f-c8c1-cb7acfd70e5d" + }, + "outputs": [], + "source": [ + "answer_alt = gemini.predict(df_image, prompt=[df_image[\"question\"], df_image[\"image\"]])\n", + "answer_alt[[\"ml_generate_text_llm_result\", \"image\"]]" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 300 + }, + "id": "KATVv2CO5RT1", + "outputId": "6ec01f27-70b6-4f69-c545-e5e3c879480c" + }, + "outputs": [], + "source": [ + "# Generate embeddings.\n", + "embed_model = llm.MultimodalEmbeddingGenerator()\n", + "embeddings = embed_model.predict(df_image[\"image\"])\n", + "embeddings" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "iRUi8AjG7cIf" + }, + "source": [ + "### 5. PDF chunking function" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "oDDuYtUm5Yiy" + }, + "outputs": [], + "source": [ + "df_pdf = bpd.from_glob_path(\"gs://cloud-samples-data/bigquery/tutorials/cymbal-pets/documents/*\", name=\"pdf\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "7jLpMYaj7nj8", + "outputId": "06d5456f-580f-4693-adff-2605104b056c" + }, + "outputs": [], + "source": [ + "df_pdf[\"chunked\"] = df_pdf[\"pdf\"].blob.pdf_chunk()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "kaPvJATN7zlw" + }, + "outputs": [], + "source": [ + "chunked = df_pdf[\"chunked\"].explode()\n", + "chunked" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [] + } + ], + "metadata": { + "colab": { + "provenance": [] + }, + "kernelspec": { + "display_name": "Python 3", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.14" + } + }, + "nbformat": 4, + "nbformat_minor": 0 +} diff --git a/noxfile.py b/noxfile.py index 3bdff699ed..5c20487ccd 100644 --- a/noxfile.py +++ b/noxfile.py @@ -784,12 +784,12 @@ def notebook(session: nox.Session): # The experimental notebooks imagine features that don't yet # exist or only exist as temporary prototypes. "notebooks/experimental/ai_operators.ipynb", - "notebooks/experimental/multimodal_dataframe.ipynb", "notebooks/experimental/semantic_operators.ipynb", # The notebooks that are added for more use cases, such as backing a # blog post, which may take longer to execute and need not be # continuously tested. "notebooks/apps/synthetic_data_generation.ipynb", + "notebooks/multimodal/multimodal_dataframe.ipynb", # too slow ] # TODO: remove exception for Python 3.13 cloud run adds a runtime for it (internal issue 333742751) From 8b30143e3320a730df168b5a72e6d18e631135ee Mon Sep 17 00:00:00 2001 From: Garrett Wu <6505921+GarrettWu@users.noreply.github.com> Date: Thu, 8 May 2025 16:42:07 -0700 Subject: [PATCH 24/36] docs: update blob_display option in snippets (#1714) --- notebooks/multimodal/multimodal_dataframe.ipynb | 6 +++--- samples/snippets/multimodal_test.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/notebooks/multimodal/multimodal_dataframe.ipynb b/notebooks/multimodal/multimodal_dataframe.ipynb index 860d8080eb..ce3f10b881 100644 --- a/notebooks/multimodal/multimodal_dataframe.ipynb +++ b/notebooks/multimodal/multimodal_dataframe.ipynb @@ -32,18 +32,18 @@ "\n", "\n", " \n", " \n", "
\n", - " \n", + " \n", " \"Colab Run in Colab\n", " \n", " \n", - " \n", + " \n", " \"GitHub\n", " View on GitHub\n", " \n", " \n", - " \n", + " \n", " \"BQ\n", " Open in BQ Studio\n", " \n", diff --git a/samples/snippets/multimodal_test.py b/samples/snippets/multimodal_test.py index fc3d60e12e..368f82d849 100644 --- a/samples/snippets/multimodal_test.py +++ b/samples/snippets/multimodal_test.py @@ -20,7 +20,7 @@ def test_multimodal_dataframe(gcs_dst_bucket: str) -> None: import bigframes # Flags to control preview image/video preview size - bigframes.options.experiments.blob_display_width = 300 + bigframes.options.display.blob_display_width = 300 import bigframes.pandas as bpd From 08d70b6ad3ab3ac7b9a57d93da00168a8de7df9a Mon Sep 17 00:00:00 2001 From: Shobhit Singh Date: Thu, 8 May 2025 16:57:00 -0700 Subject: [PATCH 25/36] docs: include import statement in the bigframes code snippet (#1699) This change improves the code snippet in https://cloud.google.com/bigquery/docs/explore-data-colab#explore_the_data --- samples/snippets/explore_query_result_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/samples/snippets/explore_query_result_test.py b/samples/snippets/explore_query_result_test.py index 42f48fd94e..7d4b241e4c 100644 --- a/samples/snippets/explore_query_result_test.py +++ b/samples/snippets/explore_query_result_test.py @@ -14,9 +14,9 @@ def test_bigquery_dataframes_explore_query_result() -> None: + # [START bigquery_dataframes_explore_query_result] import bigframes.pandas as bpd - # [START bigquery_dataframes_explore_query_result] # Load data from BigQuery query_or_table = "bigquery-public-data.ml_datasets.penguins" bq_df = bpd.read_gbq(query_or_table) From 2b3a45f8c1fd299ee97cf1c343df7c80175b4287 Mon Sep 17 00:00:00 2001 From: Shenyang Cai Date: Fri, 9 May 2025 10:03:42 -0700 Subject: [PATCH 26/36] feat: add `.dt.days`, `.dt.seconds`, `dt.microseconds`, and `dt.total_seconds()` for timedelta series. (#1713) * feat: add part accessors to timedelta series * fix doctest --- bigframes/operations/datetimes.py | 35 +++++++++ .../system/small/operations/test_datetimes.py | 44 ++++++++++++ .../pandas/core/indexes/accessor.py | 71 +++++++++++++++++++ 3 files changed, 150 insertions(+) diff --git a/bigframes/operations/datetimes.py b/bigframes/operations/datetimes.py index 032bd50390..e2439f6393 100644 --- a/bigframes/operations/datetimes.py +++ b/bigframes/operations/datetimes.py @@ -19,12 +19,18 @@ import bigframes_vendored.pandas.core.arrays.datetimelike as vendored_pandas_datetimelike import bigframes_vendored.pandas.core.indexes.accessor as vendordt +import pandas +from bigframes import dtypes from bigframes.core import log_adapter import bigframes.operations as ops import bigframes.operations.base import bigframes.series as series +_ONE_DAY = pandas.Timedelta("1d") +_ONE_SECOND = pandas.Timedelta("1s") +_ONE_MICRO = pandas.Timedelta("1us") + @log_adapter.class_logger class DatetimeMethods( @@ -80,6 +86,35 @@ def second(self) -> series.Series: def time(self) -> series.Series: return self._apply_unary_op(ops.time_op) + # Timedelta accessors + @property + def days(self) -> series.Series: + self._check_dtype(dtypes.TIMEDELTA_DTYPE) + + return self._apply_binary_op(_ONE_DAY, ops.floordiv_op) + + @property + def seconds(self) -> series.Series: + self._check_dtype(dtypes.TIMEDELTA_DTYPE) + + return self._apply_binary_op(_ONE_DAY, ops.mod_op) // _ONE_SECOND # type: ignore + + @property + def microseconds(self) -> series.Series: + self._check_dtype(dtypes.TIMEDELTA_DTYPE) + + return self._apply_binary_op(_ONE_SECOND, ops.mod_op) // _ONE_MICRO # type: ignore + + def total_seconds(self) -> series.Series: + self._check_dtype(dtypes.TIMEDELTA_DTYPE) + + return self._apply_binary_op(_ONE_SECOND, ops.div_op) + + def _check_dtype(self, target_dtype: dtypes.Dtype): + if self._dtype == target_dtype: + return + raise TypeError(f"Expect dtype: {target_dtype}, but got {self._dtype}") + @property def tz(self) -> Optional[dt.timezone]: # Assumption: pyarrow dtype diff --git a/tests/system/small/operations/test_datetimes.py b/tests/system/small/operations/test_datetimes.py index 705439fd96..69f68ca5de 100644 --- a/tests/system/small/operations/test_datetimes.py +++ b/tests/system/small/operations/test_datetimes.py @@ -30,6 +30,14 @@ ] +@pytest.fixture +def timedelta_series(session): + pd_s = pd.Series(pd.to_timedelta([1.1010101, 2.2020102, 3.3030103], unit="d")) + bf_s = session.read_pandas(pd_s) + + return bf_s, pd_s + + @pytest.mark.parametrize( ("col_name",), DATE_COLUMNS, @@ -489,3 +497,39 @@ def test_timestamp_series_diff_agg(scalars_dfs, column): expected_result = pd_series.diff() assert_series_equal(actual_result, expected_result) + + +@pytest.mark.parametrize( + "access", + [ + pytest.param(lambda x: x.dt.days, id="dt.days"), + pytest.param(lambda x: x.dt.seconds, id="dt.seconds"), + pytest.param(lambda x: x.dt.microseconds, id="dt.microseconds"), + pytest.param(lambda x: x.dt.total_seconds(), id="dt.total_seconds()"), + ], +) +def test_timedelta_dt_accessors(timedelta_series, access): + bf_s, pd_s = timedelta_series + + actual_result = access(bf_s).to_pandas() + + expected_result = access(pd_s) + assert_series_equal( + actual_result, expected_result, check_dtype=False, check_index_type=False + ) + + +@pytest.mark.parametrize( + "access", + [ + pytest.param(lambda x: x.dt.days, id="dt.days"), + pytest.param(lambda x: x.dt.seconds, id="dt.seconds"), + pytest.param(lambda x: x.dt.microseconds, id="dt.microseconds"), + pytest.param(lambda x: x.dt.total_seconds(), id="dt.total_seconds()"), + ], +) +def test_timedelta_dt_accessors_on_wrong_type_raise_exception(scalars_dfs, access): + bf_df, _ = scalars_dfs + + with pytest.raises(TypeError): + access(bf_df["timestamp_col"]) diff --git a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py index a5004c93d0..e642d11ca4 100644 --- a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py +++ b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py @@ -299,6 +299,77 @@ def year(self): raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + @property + def days(self): + """The numebr of days for each element + + **Examples:** + + >>> import pandas as pd + >>> import bigframes.pandas as bpd + >>> bpd.options.display.progress_bar = None + >>> s = bpd.Series([pd.Timedelta("4d3m2s1us")]) + >>> s + 0 4 days 00:03:02.000001 + dtype: duration[us][pyarrow] + >>> s.dt.days + 0 4 + dtype: Int64 + """ + + @property + def seconds(self): + """Number of seconds (>= 0 and less than 1 day) for each element. + + **Examples:** + + >>> import pandas as pd + >>> import bigframes.pandas as bpd + >>> bpd.options.display.progress_bar = None + >>> s = bpd.Series([pd.Timedelta("4d3m2s1us")]) + >>> s + 0 4 days 00:03:02.000001 + dtype: duration[us][pyarrow] + >>> s.dt.seconds + 0 182 + dtype: Int64 + """ + + @property + def microseconds(self): + """Number of microseconds (>= 0 and less than 1 second) for each element. + + **Examples:** + + >>> import pandas as pd + >>> import bigframes.pandas as bpd + >>> bpd.options.display.progress_bar = None + >>> s = bpd.Series([pd.Timedelta("4d3m2s1us")]) + >>> s + 0 4 days 00:03:02.000001 + dtype: duration[us][pyarrow] + >>> s.dt.microseconds + 0 1 + dtype: Int64 + """ + + def total_seconds(self): + """Return total duration of each element expressed in seconds. + + **Examples:** + + >>> import pandas as pd + >>> import bigframes.pandas as bpd + >>> bpd.options.display.progress_bar = None + >>> s = bpd.Series([pd.Timedelta("1d1m1s1us")]) + >>> s + 0 1 days 00:01:01.000001 + dtype: duration[us][pyarrow] + >>> s.dt.total_seconds() + 0 86461.000001 + dtype: Float64 + """ + @property def tz(self): """Return the timezone. From 90e277ea5e02813b0ef6a320b54a90ffba703e45 Mon Sep 17 00:00:00 2001 From: TrevorBergeron Date: Fri, 9 May 2025 11:05:28 -0700 Subject: [PATCH 27/36] refactor: Streamline executor paths (#1708) --- bigframes/core/array_value.py | 27 --- bigframes/core/compile/__init__.py | 8 +- bigframes/core/compile/api.py | 34 +--- bigframes/session/bq_caching_executor.py | 230 ++++++++++++++--------- bigframes/session/executor.py | 8 +- bigframes/session/local_scan_executor.py | 26 +-- bigframes/session/read_api_execution.py | 43 +++-- tests/system/large/test_dataframe_io.py | 18 +- tests/unit/polars_session.py | 2 +- 9 files changed, 186 insertions(+), 210 deletions(-) diff --git a/bigframes/core/array_value.py b/bigframes/core/array_value.py index 41a6882cd7..60f5315554 100644 --- a/bigframes/core/array_value.py +++ b/bigframes/core/array_value.py @@ -172,33 +172,6 @@ def order_ambiguous(self) -> bool: def supports_fast_peek(self) -> bool: return bigframes.core.tree_properties.can_fast_peek(self.node) - def as_cached( - self: ArrayValue, - cache_table: google.cloud.bigquery.Table, - ordering: Optional[orderings.RowOrdering], - ) -> ArrayValue: - """ - Replace the node with an equivalent one that references a table where the value has been materialized to. - """ - table = nodes.GbqTable.from_table(cache_table) - source = nodes.BigqueryDataSource( - table, ordering=ordering, n_rows=cache_table.num_rows - ) - # Assumption: GBQ cached table uses field name as bq column name - scan_list = nodes.ScanList( - tuple( - nodes.ScanItem(field.id, field.dtype, field.id.name) - for field in self.node.fields - ) - ) - node = nodes.CachedTableNode( - original_node=self.node, - source=source, - table_session=self.session, - scan_list=scan_list, - ) - return ArrayValue(node) - def get_column_type(self, key: str) -> bigframes.dtypes.Dtype: return self.schema.get_type(key) diff --git a/bigframes/core/compile/__init__.py b/bigframes/core/compile/__init__.py index 0bfdf2222d..e2487306ab 100644 --- a/bigframes/core/compile/__init__.py +++ b/bigframes/core/compile/__init__.py @@ -13,9 +13,13 @@ # limitations under the License. from __future__ import annotations -from bigframes.core.compile.api import SQLCompiler, test_only_ibis_inferred_schema +from bigframes.core.compile.api import test_only_ibis_inferred_schema +from bigframes.core.compile.compiler import compile_sql +from bigframes.core.compile.configs import CompileRequest, CompileResult __all__ = [ - "SQLCompiler", "test_only_ibis_inferred_schema", + "compile_sql", + "CompileRequest", + "CompileResult", ] diff --git a/bigframes/core/compile/api.py b/bigframes/core/compile/api.py index 287549b571..ddd8622327 100644 --- a/bigframes/core/compile/api.py +++ b/bigframes/core/compile/api.py @@ -13,43 +13,13 @@ # limitations under the License. from __future__ import annotations -from typing import Optional, Sequence, Tuple, TYPE_CHECKING - -import google.cloud.bigquery as bigquery +from typing import TYPE_CHECKING from bigframes.core import rewrite -from bigframes.core.compile import compiler, configs +from bigframes.core.compile import compiler if TYPE_CHECKING: import bigframes.core.nodes - import bigframes.core.ordering - - -class SQLCompiler: - def compile( - self, - node: bigframes.core.nodes.BigFrameNode, - *, - ordered: bool = True, - limit: Optional[int] = None, - ) -> str: - """Compile node into sql where rows are sorted with ORDER BY.""" - request = configs.CompileRequest(node, sort_rows=ordered, peek_count=limit) - return compiler.compile_sql(request).sql - - def compile_raw( - self, - node: bigframes.core.nodes.BigFrameNode, - ) -> Tuple[ - str, Sequence[bigquery.SchemaField], bigframes.core.ordering.RowOrdering - ]: - """Compile node into sql that exposes all columns, including hidden ordering-only columns.""" - request = configs.CompileRequest( - node, sort_rows=False, materialize_all_order_keys=True - ) - result = compiler.compile_sql(request) - assert result.row_order is not None - return result.sql, result.sql_schema, result.row_order def test_only_ibis_inferred_schema(node: bigframes.core.nodes.BigFrameNode): diff --git a/bigframes/session/bq_caching_executor.py b/bigframes/session/bq_caching_executor.py index 533f49b666..72f2dfa4b5 100644 --- a/bigframes/session/bq_caching_executor.py +++ b/bigframes/session/bq_caching_executor.py @@ -14,6 +14,7 @@ from __future__ import annotations +import dataclasses import math import os from typing import cast, Literal, Mapping, Optional, Sequence, Tuple, Union @@ -27,8 +28,7 @@ import google.cloud.bigquery_storage_v1 import bigframes.core -from bigframes.core import rewrite -import bigframes.core.compile +from bigframes.core import compile, rewrite import bigframes.core.guid import bigframes.core.nodes as nodes import bigframes.core.ordering as order @@ -50,6 +50,60 @@ MAX_SMALL_RESULT_BYTES = 10 * 1024 * 1024 * 1024 # 10G +@dataclasses.dataclass +class OutputSpec: + require_bq_table: bool + cluster_cols: tuple[str, ...] + + def with_require_table(self, value: bool) -> OutputSpec: + return dataclasses.replace(self, require_bq_table=value) + + +def _get_default_output_spec() -> OutputSpec: + return OutputSpec( + require_bq_table=bigframes.options.bigquery.allow_large_results, cluster_cols=() + ) + + +class ExecutionCache: + def __init__(self): + # current assumption is only 1 cache of a given node + # in future, might have multiple caches, with different layout, localities + self._cached_executions: weakref.WeakKeyDictionary[ + nodes.BigFrameNode, nodes.BigFrameNode + ] = weakref.WeakKeyDictionary() + + @property + def mapping(self) -> Mapping[nodes.BigFrameNode, nodes.BigFrameNode]: + return self._cached_executions + + def cache_results_table( + self, + original_root: nodes.BigFrameNode, + table: bigquery.Table, + ordering: order.RowOrdering, + ): + # Assumption: GBQ cached table uses field name as bq column name + scan_list = nodes.ScanList( + tuple( + nodes.ScanItem(field.id, field.dtype, field.id.sql) + for field in original_root.fields + ) + ) + cached_replacement = nodes.CachedTableNode( + source=nodes.BigqueryDataSource( + nodes.GbqTable.from_table(table), + ordering=ordering, + n_rows=table.num_rows, + ), + scan_list=scan_list, + table_session=original_root.session, + original_node=original_root, + ) + assert original_root.schema == cached_replacement.schema + self._cached_executions[original_root] = cached_replacement + + class BigQueryCachingExecutor(executor.Executor): """Computes BigFrames values using BigQuery Engine. @@ -70,13 +124,8 @@ def __init__( ): self.bqclient = bqclient self.storage_manager = storage_manager - self.compiler: bigframes.core.compile.SQLCompiler = ( - bigframes.core.compile.SQLCompiler() - ) self.strictly_ordered: bool = strictly_ordered - self._cached_executions: weakref.WeakKeyDictionary[ - nodes.BigFrameNode, nodes.BigFrameNode - ] = weakref.WeakKeyDictionary() + self.cache: ExecutionCache = ExecutionCache() self.metrics = metrics self.bqstoragereadclient = bqstoragereadclient # Simple left-to-right precedence for now @@ -98,7 +147,8 @@ def to_sql( if offset_column: array_value, _ = array_value.promote_offsets() node = self.logical_plan(array_value.node) if enable_cache else array_value.node - return self.compiler.compile(node, ordered=ordered) + compiled = compile.compile_sql(compile.CompileRequest(node, sort_rows=ordered)) + return compiled.sql def execute( self, @@ -113,19 +163,37 @@ def execute( if bigframes.options.compute.enable_multi_query_execution: self._simplify_with_caching(array_value) + output_spec = _get_default_output_spec() + if use_explicit_destination is not None: + output_spec = output_spec.with_require_table(use_explicit_destination) + plan = self.logical_plan(array_value.node) - # Use explicit destination to avoid 10GB limit of temporary table - destination_table = ( - self.storage_manager.create_temp_table( - array_value.schema.to_bigquery(), cluster_cols=[] - ) - if use_explicit_destination - else None - ) return self._execute_plan( plan, ordered=ordered, - destination=destination_table, + output_spec=output_spec, + ) + + def peek( + self, + array_value: bigframes.core.ArrayValue, + n_rows: int, + use_explicit_destination: Optional[bool] = None, + ) -> executor.ExecuteResult: + """ + A 'peek' efficiently accesses a small number of rows in the dataframe. + """ + plan = self.logical_plan(array_value.node) + if not tree_properties.can_fast_peek(plan): + msg = bfe.format_message("Peeking this value cannot be done efficiently.") + warnings.warn(msg) + + output_spec = _get_default_output_spec() + if use_explicit_destination is not None: + output_spec = output_spec.with_require_table(use_explicit_destination) + + return self._execute_plan( + plan, ordered=False, output_spec=output_spec, peek=n_rows ) def export_gbq( @@ -211,34 +279,6 @@ def dry_run( query_job = self.bqclient.query(sql, job_config=job_config) return query_job - def peek( - self, - array_value: bigframes.core.ArrayValue, - n_rows: int, - use_explicit_destination: Optional[bool] = None, - ) -> executor.ExecuteResult: - """ - A 'peek' efficiently accesses a small number of rows in the dataframe. - """ - plan = self.logical_plan(array_value.node) - if not tree_properties.can_fast_peek(plan): - msg = bfe.format_message("Peeking this value cannot be done efficiently.") - warnings.warn(msg) - if use_explicit_destination is None: - use_explicit_destination = bigframes.options.bigquery.allow_large_results - - destination_table = ( - self.storage_manager.create_temp_table( - array_value.schema.to_bigquery(), cluster_cols=[] - ) - if use_explicit_destination - else None - ) - - return self._execute_plan( - plan, ordered=False, destination=destination_table, peek=n_rows - ) - def cached( self, array_value: bigframes.core.ArrayValue, *, config: executor.CacheConfig ) -> None: @@ -311,7 +351,7 @@ def _run_execute_query( raise def replace_cached_subtrees(self, node: nodes.BigFrameNode) -> nodes.BigFrameNode: - return nodes.top_down(node, lambda x: self._cached_executions.get(x, x)) + return nodes.top_down(node, lambda x: self.cache.mapping.get(x, x)) def _is_trivially_executable(self, array_value: bigframes.core.ArrayValue): """ @@ -337,37 +377,39 @@ def _cache_with_cluster_cols( self, array_value: bigframes.core.ArrayValue, cluster_cols: Sequence[str] ): """Executes the query and uses the resulting table to rewrite future executions.""" - - sql, schema, ordering_info = self.compiler.compile_raw( - self.logical_plan(array_value.node) + plan = self.logical_plan(array_value.node) + compiled = compile.compile_sql( + compile.CompileRequest( + plan, sort_rows=False, materialize_all_order_keys=True + ) ) - tmp_table = self._sql_as_cached_temp_table( - sql, - schema, - cluster_cols=bq_io.select_cluster_cols(schema, cluster_cols), + tmp_table_ref = self._sql_as_cached_temp_table( + compiled.sql, + compiled.sql_schema, + cluster_cols=bq_io.select_cluster_cols(compiled.sql_schema, cluster_cols), ) - cached_replacement = array_value.as_cached( - cache_table=self.bqclient.get_table(tmp_table), - ordering=ordering_info, - ).node - self._cached_executions[array_value.node] = cached_replacement + tmp_table = self.bqclient.get_table(tmp_table_ref) + assert compiled.row_order is not None + self.cache.cache_results_table(array_value.node, tmp_table, compiled.row_order) def _cache_with_offsets(self, array_value: bigframes.core.ArrayValue): """Executes the query and uses the resulting table to rewrite future executions.""" offset_column = bigframes.core.guid.generate_guid("bigframes_offsets") w_offsets, offset_column = array_value.promote_offsets() - sql = self.compiler.compile(self.logical_plan(w_offsets.node), ordered=False) - - tmp_table = self._sql_as_cached_temp_table( - sql, - w_offsets.schema.to_bigquery(), + compiled = compile.compile_sql( + compile.CompileRequest( + self.logical_plan(w_offsets.node), + sort_rows=False, + ) + ) + tmp_table_ref = self._sql_as_cached_temp_table( + compiled.sql, + compiled.sql_schema, cluster_cols=[offset_column], ) - cached_replacement = array_value.as_cached( - cache_table=self.bqclient.get_table(tmp_table), - ordering=order.TotalOrdering.from_offset_col(offset_column), - ).node - self._cached_executions[array_value.node] = cached_replacement + tmp_table = self.bqclient.get_table(tmp_table_ref) + assert compiled.row_order is not None + self.cache.cache_results_table(array_value.node, tmp_table, compiled.row_order) def _cache_with_session_awareness( self, @@ -408,7 +450,7 @@ def _cache_most_complex_subtree(self, node: nodes.BigFrameNode) -> bool: node, min_complexity=(QUERY_COMPLEXITY_LIMIT / 500), max_complexity=QUERY_COMPLEXITY_LIMIT, - cache=dict(self._cached_executions), + cache=dict(self.cache.mapping), # Heuristic: subtree_compleixty * (copies of subtree)^2 heuristic=lambda complexity, count: math.log(complexity) + 2 * math.log(count), @@ -450,7 +492,7 @@ def _validate_result_schema( bq_schema: list[bigquery.SchemaField], ): actual_schema = _sanitize(tuple(bq_schema)) - ibis_schema = bigframes.core.compile.test_only_ibis_inferred_schema( + ibis_schema = compile.test_only_ibis_inferred_schema( self.logical_plan(array_value.node) ).to_bigquery() internal_schema = _sanitize(array_value.schema.to_bigquery()) @@ -471,48 +513,46 @@ def _execute_plan( self, plan: nodes.BigFrameNode, ordered: bool, - destination: Optional[bq_table.TableReference] = None, + output_spec: OutputSpec, peek: Optional[int] = None, - ): + ) -> executor.ExecuteResult: """Just execute whatever plan as is, without further caching or decomposition.""" - # First try to execute fast-paths - if (not destination) and (not peek): + if not output_spec.require_bq_table: for semi_executor in self._semi_executors: - maybe_result = semi_executor.execute(plan, ordered=ordered) + maybe_result = semi_executor.execute(plan, ordered=ordered, peek=peek) if maybe_result: return maybe_result + # Use explicit destination to avoid 10GB limit of temporary table + destination_table = ( + self.storage_manager.create_temp_table( + plan.schema.to_bigquery(), cluster_cols=output_spec.cluster_cols + ) + if output_spec.require_bq_table + else None + ) + # TODO(swast): plumb through the api_name of the user-facing api that # caused this query. job_config = bigquery.QueryJobConfig() # Use explicit destination to avoid 10GB limit of temporary table - if destination is not None: - job_config.destination = destination - sql = self.compiler.compile(plan, ordered=ordered, limit=peek) + if destination_table is not None: + job_config.destination = destination_table + compiled = compile.compile_sql( + compile.CompileRequest(plan, sort_rows=ordered, peek_count=peek) + ) iterator, query_job = self._run_execute_query( - sql=sql, + sql=compiled.sql, job_config=job_config, - query_with_job=(destination is not None), + query_with_job=(destination_table is not None), ) - # Though we provide the read client, iterator may or may not use it based on what is efficient for the result - def iterator_supplier(): - return iterator.to_arrow_iterable(bqstorage_client=self.bqstoragereadclient) - if query_job: size_bytes = self.bqclient.get_table(query_job.destination).num_bytes else: size_bytes = None - if size_bytes is not None and size_bytes >= MAX_SMALL_RESULT_BYTES: - msg = bfe.format_message( - "The query result size has exceeded 10 GB. In BigFrames 2.0 and " - "later, you might need to manually set `allow_large_results=True` in " - "the IO method or adjust the BigFrames option: " - "`bigframes.options.bigquery.allow_large_results=True`." - ) - warnings.warn(msg, FutureWarning) # Runs strict validations to ensure internal type predictions and ibis are completely in sync # Do not execute these validations outside of testing suite. if "PYTEST_CURRENT_TEST" in os.environ: @@ -521,7 +561,9 @@ def iterator_supplier(): ) return executor.ExecuteResult( - arrow_batches=iterator_supplier, + arrow_batches=iterator.to_arrow_iterable( + bqstorage_client=self.bqstoragereadclient + ), schema=plan.schema, query_job=query_job, total_bytes=size_bytes, diff --git a/bigframes/session/executor.py b/bigframes/session/executor.py index fd9d0a083f..ee1218017b 100644 --- a/bigframes/session/executor.py +++ b/bigframes/session/executor.py @@ -18,7 +18,7 @@ import dataclasses import functools import itertools -from typing import Callable, Iterator, Literal, Mapping, Optional, Sequence, Union +from typing import Iterator, Literal, Mapping, Optional, Sequence, Union from google.cloud import bigquery import pandas as pd @@ -32,7 +32,7 @@ @dataclasses.dataclass(frozen=True) class ExecuteResult: - arrow_batches: Callable[[], Iterator[pyarrow.RecordBatch]] + arrow_batches: Iterator[pyarrow.RecordBatch] schema: bigframes.core.schema.ArraySchema query_job: Optional[bigquery.QueryJob] = None total_bytes: Optional[int] = None @@ -42,7 +42,7 @@ def to_arrow_table(self) -> pyarrow.Table: # Need to provide schema if no result rows, as arrow can't infer # If ther are rows, it is safest to infer schema from batches. # Any discrepencies between predicted schema and actual schema will produce errors. - batches = iter(self.arrow_batches()) + batches = iter(self.arrow_batches) peek_it = itertools.islice(batches, 0, 1) peek_value = list(peek_it) # TODO: Enforce our internal schema on the table for consistency @@ -63,7 +63,7 @@ def to_pandas_batches( assert (max_results is None) or (max_results > 0) batch_iter: Iterator[ Union[pyarrow.Table, pyarrow.RecordBatch] - ] = self.arrow_batches() + ] = self.arrow_batches if max_results is not None: batch_iter = pyarrow_utils.truncate_pyarrow_iterable( batch_iter, max_results diff --git a/bigframes/session/local_scan_executor.py b/bigframes/session/local_scan_executor.py index 67e381ab8a..88304fa181 100644 --- a/bigframes/session/local_scan_executor.py +++ b/bigframes/session/local_scan_executor.py @@ -35,30 +35,24 @@ def execute( return None # TODO: Can support some slicing, sorting - def iterator_supplier(): - offsets_col = ( - node.offsets_col.sql if (node.offsets_col is not None) else None - ) - arrow_table = node.local_data_source.to_pyarrow_table( - offsets_col=offsets_col - ) - if peek: - arrow_table = arrow_table.slice(0, peek) + offsets_col = node.offsets_col.sql if (node.offsets_col is not None) else None + arrow_table = node.local_data_source.to_pyarrow_table(offsets_col=offsets_col) + if peek: + arrow_table = arrow_table.slice(0, peek) - needed_cols = [item.source_id for item in node.scan_list.items] - if offsets_col is not None: - needed_cols.append(offsets_col) + needed_cols = [item.source_id for item in node.scan_list.items] + if offsets_col is not None: + needed_cols.append(offsets_col) - arrow_table = arrow_table.select(needed_cols) - arrow_table = arrow_table.rename_columns([id.sql for id in node.ids]) - yield from arrow_table.to_batches() + arrow_table = arrow_table.select(needed_cols) + arrow_table = arrow_table.rename_columns([id.sql for id in node.ids]) total_rows = node.row_count if (peek is not None) and (total_rows is not None): total_rows = min(peek, total_rows) return executor.ExecuteResult( - arrow_batches=iterator_supplier, + arrow_batches=arrow_table.to_batches(), schema=plan.schema, query_job=None, total_bytes=None, diff --git a/bigframes/session/read_api_execution.py b/bigframes/session/read_api_execution.py index ae1272e722..46d55eb303 100644 --- a/bigframes/session/read_api_execution.py +++ b/bigframes/session/read_api_execution.py @@ -13,7 +13,7 @@ # limitations under the License. from __future__ import annotations -from typing import Any, Optional +from typing import Any, Iterator, Optional from google.cloud import bigquery_storage_v1 import pyarrow as pa @@ -66,26 +66,25 @@ def execute( table_mod_options["snapshot_time"] = snapshot_time = snapshot_time table_mods = bq_storage_types.ReadSession.TableModifiers(**table_mod_options) - def iterator_supplier(): - requested_session = bq_storage_types.stream.ReadSession( - table=bq_table.to_bqstorage(), - data_format=bq_storage_types.DataFormat.ARROW, - read_options=read_options, - table_modifiers=table_mods, - ) - # Single stream to maintain ordering - request = bq_storage_types.CreateReadSessionRequest( - parent=f"projects/{self.project}", - read_session=requested_session, - max_stream_count=1, - ) - session = self.bqstoragereadclient.create_read_session( - request=request, retry=None - ) - - if not session.streams: - return iter([]) + requested_session = bq_storage_types.stream.ReadSession( + table=bq_table.to_bqstorage(), + data_format=bq_storage_types.DataFormat.ARROW, + read_options=read_options, + table_modifiers=table_mods, + ) + # Single stream to maintain ordering + request = bq_storage_types.CreateReadSessionRequest( + parent=f"projects/{self.project}", + read_session=requested_session, + max_stream_count=1, + ) + session = self.bqstoragereadclient.create_read_session( + request=request, retry=None + ) + if not session.streams: + batches: Iterator[pa.RecordBatch] = iter([]) + else: reader = self.bqstoragereadclient.read_rows( session.streams[0].name, retry=None ) @@ -97,10 +96,10 @@ def process_page(page): pa_batch.columns, names=[id.sql for id in node.ids] ) - return map(process_page, rowstream.pages) + batches = map(process_page, rowstream.pages) return executor.ExecuteResult( - arrow_batches=iterator_supplier, + arrow_batches=batches, schema=plan.schema, query_job=None, total_bytes=None, diff --git a/tests/system/large/test_dataframe_io.py b/tests/system/large/test_dataframe_io.py index ee9daa4e31..b90139b12f 100644 --- a/tests/system/large/test_dataframe_io.py +++ b/tests/system/large/test_dataframe_io.py @@ -12,9 +12,6 @@ # See the License for the specific language governing permissions and # limitations under the License. - -import warnings - import google.api_core.exceptions import pytest @@ -37,16 +34,13 @@ def test_to_pandas_batches_override_global_option( ): with bigframes.option_context(LARGE_TABLE_OPTION, False): df = session.read_gbq(WIKIPEDIA_TABLE) - with warnings.catch_warnings(record=True) as w: - warnings.simplefilter("always") - next( - df.to_pandas_batches( - page_size=500, max_results=1500, allow_large_results=True - ) + pages = list( + df.to_pandas_batches( + page_size=500, max_results=1500, allow_large_results=True ) - assert len(w) == 1 - assert issubclass(w[0].category, FutureWarning) - assert "The query result size has exceeded 10 GB." in str(w[0].message) + ) + assert all((len(page) <= 500) for page in pages) + assert sum(len(page) for page in pages) == 1500 def test_to_pandas_raise_when_large_result_not_allowed(session): diff --git a/tests/unit/polars_session.py b/tests/unit/polars_session.py index a27db0e438..d592b49038 100644 --- a/tests/unit/polars_session.py +++ b/tests/unit/polars_session.py @@ -51,7 +51,7 @@ def execute( # Currently, pyarrow types might not quite be exactly the ones in the bigframes schema. # Nullability may be different, and might use large versions of list, string datatypes. return bigframes.session.executor.ExecuteResult( - arrow_batches=lambda: pa_table.to_batches(), + arrow_batches=pa_table.to_batches(), schema=array_value.schema, total_bytes=pa_table.nbytes, total_rows=pa_table.num_rows, From f5d91f3995d816b7c524ed43ec27d9bb3f54f7fb Mon Sep 17 00:00:00 2001 From: Shuowei Li Date: Fri, 9 May 2025 11:40:11 -0700 Subject: [PATCH 28/36] chore: change error to warning for gemini models (#1715) * chore: change error to warning for gemini models * remove model --- bigframes/ml/llm.py | 21 +++++++++++++++++---- bigframes/ml/loader.py | 1 + 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/bigframes/ml/llm.py b/bigframes/ml/llm.py index 111ad20f8a..f93ba8b720 100644 --- a/bigframes/ml/llm.py +++ b/bigframes/ml/llm.py @@ -53,6 +53,7 @@ _GEMINI_2_FLASH_EXP_ENDPOINT = "gemini-2.0-flash-exp" _GEMINI_2_FLASH_001_ENDPOINT = "gemini-2.0-flash-001" _GEMINI_2_FLASH_LITE_001_ENDPOINT = "gemini-2.0-flash-lite-001" +_GEMINI_2P5_PRO_PREVIEW_ENDPOINT = "gemini-2.5-pro-preview-05-06" _GEMINI_ENDPOINTS = ( _GEMINI_1P5_PRO_PREVIEW_ENDPOINT, _GEMINI_1P5_PRO_FLASH_PREVIEW_ENDPOINT, @@ -104,6 +105,12 @@ _REMOVE_DEFAULT_MODEL_WARNING = "Since upgrading the default model can cause unintended breakages, the default model will be removed in BigFrames 3.0. Please supply an explicit model to avoid this message." +_GEMINI_MULTIMODAL_MODEL_NOT_SUPPORTED_WARNING = ( + "The model '{model_name}' may not be fully supported by GeminiTextGenerator for Multimodal prompts. " + "GeminiTextGenerator is known to support the following models for Multimodal prompts: {known_models}. " + "If you proceed with '{model_name}', it might not work as expected or could lead to errors with multimodal inputs." +) + @log_adapter.class_logger class TextEmbeddingGenerator(base.RetriableRemotePredictor): @@ -540,9 +547,10 @@ def fit( GeminiTextGenerator: Fitted estimator. """ if self.model_name not in _GEMINI_FINE_TUNE_SCORE_ENDPOINTS: - raise NotImplementedError( + msg = exceptions.format_message( "fit() only supports gemini-1.5-pro-002, or gemini-1.5-flash-002 model." ) + warnings.warn(msg) X, y = utils.batch_convert_to_dataframe(X, y) @@ -651,9 +659,13 @@ def predict( if prompt: if self.model_name not in _GEMINI_MULTIMODAL_ENDPOINTS: - raise NotImplementedError( - f"GeminiTextGenerator only supports model_name {', '.join(_GEMINI_MULTIMODAL_ENDPOINTS)} for Multimodal prompt." + msg = exceptions.format_message( + _GEMINI_MULTIMODAL_MODEL_NOT_SUPPORTED_WARNING.format( + model_name=self.model_name, + known_models=", ".join(_GEMINI_MULTIMODAL_ENDPOINTS), + ) ) + warnings.warn(msg) df_prompt = X[[X.columns[0]]].rename( columns={X.columns[0]: "bigframes_placeholder_col"} @@ -750,9 +762,10 @@ def score( raise RuntimeError("A model must be fitted before score") if self.model_name not in _GEMINI_FINE_TUNE_SCORE_ENDPOINTS: - raise NotImplementedError( + msg = exceptions.format_message( "score() only supports gemini-1.5-pro-002, gemini-1.5-flash-2, gemini-2.0-flash-001, and gemini-2.0-flash-lite-001 model." ) + warnings.warn(msg) X, y = utils.batch_convert_to_dataframe(X, y, session=self._bqml_model.session) diff --git a/bigframes/ml/loader.py b/bigframes/ml/loader.py index 83c665a50b..a6366273fe 100644 --- a/bigframes/ml/loader.py +++ b/bigframes/ml/loader.py @@ -66,6 +66,7 @@ llm._GEMINI_2_FLASH_EXP_ENDPOINT: llm.GeminiTextGenerator, llm._GEMINI_2_FLASH_001_ENDPOINT: llm.GeminiTextGenerator, llm._GEMINI_2_FLASH_LITE_001_ENDPOINT: llm.GeminiTextGenerator, + llm._GEMINI_2P5_PRO_PREVIEW_ENDPOINT: llm.GeminiTextGenerator, llm._CLAUDE_3_HAIKU_ENDPOINT: llm.Claude3TextGenerator, llm._CLAUDE_3_SONNET_ENDPOINT: llm.Claude3TextGenerator, llm._CLAUDE_3_5_SONNET_ENDPOINT: llm.Claude3TextGenerator, From c260fc86c850fc12eee78be7af175cc395224823 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tim=20Swe=C3=B1a=20=28Swast=29?= Date: Fri, 9 May 2025 14:38:24 -0500 Subject: [PATCH 29/36] chore: add pyformat_args to _read_gbq_colab (#1704) * chore: add private _read_gbq_colab method that uses partial ordering mode, disables progress bars, disables default index, and communicates via callbacks * add colab read gbq * add test for ordering * add ordered argument to to_pandas_batches * add unit test looking for job labels * remove ordered option for to_pandas_batches * ignore type for mock job configs * chore: add pyformat_args to _read_gbq_colab * fix unit tests * add test for _read_gbq_colab * escape strings * fix null support --- bigframes/core/pyformat.py | 111 ++++++++++++++ bigframes/core/sql.py | 18 ++- bigframes/session/__init__.py | 27 +++- .../small/session/test_read_gbq_colab.py | 66 ++++++++ tests/unit/core/test_pyformat.py | 145 ++++++++++++++++++ 5 files changed, 362 insertions(+), 5 deletions(-) create mode 100644 bigframes/core/pyformat.py create mode 100644 tests/unit/core/test_pyformat.py diff --git a/bigframes/core/pyformat.py b/bigframes/core/pyformat.py new file mode 100644 index 0000000000..98f175d300 --- /dev/null +++ b/bigframes/core/pyformat.py @@ -0,0 +1,111 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +"""Helpers for the pyformat feature.""" + +# TODO(tswast): consolidate with pandas-gbq and bigquery-magics. See: +# https://github.com/googleapis/python-bigquery-magics/blob/main/bigquery_magics/pyformat.py + +from __future__ import annotations + +import string +import typing +from typing import Any, Union + +import google.cloud.bigquery +import google.cloud.bigquery.table + +_BQ_TABLE_TYPES = Union[ + google.cloud.bigquery.Table, + google.cloud.bigquery.TableReference, + google.cloud.bigquery.table.TableListItem, +] + + +def _table_to_sql(table: _BQ_TABLE_TYPES) -> str: + return f"`{table.project}`.`{table.dataset_id}`.`{table.table_id}`" + + +def _field_to_template_value(name: str, value: Any) -> str: + """Convert value to something embeddable in a SQL string.""" + import bigframes.core.sql # Avoid circular imports + + _validate_type(name, value) + + table_types = typing.get_args(_BQ_TABLE_TYPES) + if isinstance(value, table_types): + return _table_to_sql(value) + + # TODO(tswast): convert DataFrame objects to gbq tables or a literals subquery. + return bigframes.core.sql.simple_literal(value) + + +def _validate_type(name: str, value: Any): + """Raises TypeError if value is unsupported.""" + import bigframes.core.sql # Avoid circular imports + + if value is None: + return # None can't be used in isinstance, but is a valid literal. + + supported_types = typing.get_args(_BQ_TABLE_TYPES) + typing.get_args( + bigframes.core.sql.SIMPLE_LITERAL_TYPES + ) + if not isinstance(value, supported_types): + raise TypeError( + f"{name} has unsupported type: {type(value)}. " + f"Only {supported_types} are supported." + ) + + +def _parse_fields(sql_template: str) -> list[str]: + return [ + field_name + for _, field_name, _, _ in string.Formatter().parse(sql_template) + if field_name is not None + ] + + +def pyformat( + sql_template: str, + *, + pyformat_args: dict, + # TODO: add dry_run parameter to avoid expensive API calls in conversion + # TODO: and session to upload data / convert to table if necessary +) -> str: + """Unsafe Python-style string formatting of SQL string. + + Only some data types supported. + + Warning: strings are **not** escaped. This allows them to be used in + contexts such as table identifiers, where normal query parameters are not + supported. + + Args: + sql_template (str): + SQL string with 0+ {var_name}-style format options. + pyformat_args (dict): + Variable namespace to use for formatting. + + Raises: + TypeError: if a referenced variable is not of a supported type. + KeyError: if a referenced variable is not found. + """ + fields = _parse_fields(sql_template) + + format_kwargs = {} + for name in fields: + value = pyformat_args[name] + format_kwargs[name] = _field_to_template_value(name, value) + + return sql_template.format(**format_kwargs) diff --git a/bigframes/core/sql.py b/bigframes/core/sql.py index 04e678e713..ccd2a16ddc 100644 --- a/bigframes/core/sql.py +++ b/bigframes/core/sql.py @@ -42,10 +42,22 @@ to_wkt = dumps +SIMPLE_LITERAL_TYPES = Union[ + bytes, + str, + int, + bool, + float, + datetime.datetime, + datetime.date, + datetime.time, + decimal.Decimal, + list, +] + + ### Writing SQL Values (literals, column references, table references, etc.) -def simple_literal( - value: bytes | str | int | bool | float | datetime.datetime | list | None, -): +def simple_literal(value: Union[SIMPLE_LITERAL_TYPES, None]) -> str: """Return quoted input string.""" # https://cloud.google.com/bigquery/docs/reference/standard-sql/lexical#literals diff --git a/bigframes/session/__init__.py b/bigframes/session/__init__.py index 7260553c14..b54dc1d691 100644 --- a/bigframes/session/__init__.py +++ b/bigframes/session/__init__.py @@ -61,6 +61,7 @@ import bigframes._config.bigquery_options as bigquery_options import bigframes.clients from bigframes.core import blocks +import bigframes.core.pyformat # Even though the ibis.backends.bigquery import is unused, it's needed # to register new and replacement ops with the Ibis BigQuery backend. @@ -480,16 +481,38 @@ def _read_gbq_colab( self, query: str, # TODO: Add a callback parameter that takes some kind of Event object. - # TODO: Add parameter for variables for string formatting. # TODO: Add dry_run parameter. + *, + pyformat_args: Optional[Dict[str, Any]] = None, ) -> dataframe.DataFrame: """A version of read_gbq that has the necessary default values for use in colab integrations. This includes, no ordering, no index, no progress bar, always use string formatting for embedding local variables / dataframes. + + Args: + query (str): + A SQL query string to execute. Results (if any) are turned into + a DataFrame. + pyformat_args (dict): + A dictionary of potential variables to replace in ``query``. + Note: strings are _not_ escaped. Use query parameters for these, + instead. Note: unlike read_gbq / read_gbq_query, even if set to + None, this function always assumes {var} refers to a variable + that is supposed to be supplied in this dictionary. """ + # TODO: Allow for a table ID to avoid queries like with read_gbq? + + if pyformat_args is None: + pyformat_args = {} + + # TODO: move this to read_gbq_query if/when we expose this feature + # beyond in _read_gbq_colab. + query = bigframes.core.pyformat.pyformat( + query, + pyformat_args=pyformat_args, + ) - # TODO: Allow for a table ID to avoid queries like read_gbq? return self._loader.read_gbq_query( query=query, index_col=bigframes.enums.DefaultIndexKind.NULL, diff --git a/tests/system/small/session/test_read_gbq_colab.py b/tests/system/small/session/test_read_gbq_colab.py index 7ade85b2b2..54fdd4014e 100644 --- a/tests/system/small/session/test_read_gbq_colab.py +++ b/tests/system/small/session/test_read_gbq_colab.py @@ -14,6 +14,9 @@ """System tests for read_gbq_colab helper functions.""" +import pandas +import pandas.testing + def test_read_gbq_colab_to_pandas_batches_preserves_order_by(maybe_ordered_session): df = maybe_ordered_session._read_gbq_colab( @@ -39,3 +42,66 @@ def test_read_gbq_colab_to_pandas_batches_preserves_order_by(maybe_ordered_sessi total_rows += len(batch.index) assert total_rows > 0 + + +def test_read_gbq_colab_includes_formatted_scalars(session): + pyformat_args = { + "some_integer": 123, + "some_string": "This could be dangerous, but we esape it", + # This is not a supported type, but ignored if not referenced. + "some_object": object(), + } + df = session._read_gbq_colab( + """ + SELECT {some_integer} as some_integer, + {some_string} as some_string, + '{{escaped}}' as escaped + """, + pyformat_args=pyformat_args, + ) + result = df.to_pandas() + pandas.testing.assert_frame_equal( + result, + pandas.DataFrame( + { + "some_integer": pandas.Series([123], dtype=pandas.Int64Dtype()), + "some_string": pandas.Series( + ["This could be dangerous, but we esape it"], + dtype="string[pyarrow]", + ), + "escaped": pandas.Series(["{escaped}"], dtype="string[pyarrow]"), + } + ), + ) + + +def test_read_gbq_colab_includes_formatted_bigframes_dataframe(session): + pyformat_args = { + # TODO: put a bigframes DataFrame here. + "some_integer": 123, + "some_string": "This could be dangerous, but we esape it", + # This is not a supported type, but ignored if not referenced. + "some_object": object(), + } + df = session._read_gbq_colab( + """ + SELECT {some_integer} as some_integer, + {some_string} as some_string, + '{{escaped}}' as escaped + """, + pyformat_args=pyformat_args, + ) + result = df.to_pandas() + pandas.testing.assert_frame_equal( + result, + pandas.DataFrame( + { + "some_integer": pandas.Series([123], dtype=pandas.Int64Dtype()), + "some_string": pandas.Series( + ["This could be dangerous, but we esape it"], + dtype="string[pyarrow]", + ), + "escaped": pandas.Series(["{escaped}"], dtype="string[pyarrow]"), + } + ), + ) diff --git a/tests/unit/core/test_pyformat.py b/tests/unit/core/test_pyformat.py new file mode 100644 index 0000000000..466f3d6116 --- /dev/null +++ b/tests/unit/core/test_pyformat.py @@ -0,0 +1,145 @@ +# Copyright 2025 Google LLC +# +# Licensed 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 +# +# https://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. + +"""Tests for the pyformat feature.""" + +# TODO(tswast): consolidate with pandas-gbq and bigquery-magics. See: +# https://github.com/googleapis/python-bigquery-magics/blob/main/tests/unit/bigquery/test_pyformat.py + +from __future__ import annotations + +from typing import Any, Dict, List + +import google.cloud.bigquery +import google.cloud.bigquery.table +import pytest + +import bigframes.core.pyformat as pyformat + + +@pytest.mark.parametrize( + ("sql_template", "expected"), + ( + ( + "{my_project}.{my_dataset}.{my_table}", + ["my_project", "my_dataset", "my_table"], + ), + ( + "{{not a format variable}}", + [], + ), + ), +) +def test_parse_fields(sql_template: str, expected: List[str]): + fields = pyformat._parse_fields(sql_template) + fields.sort() + expected.sort() + assert fields == expected + + +def test_pyformat_with_unsupported_type_raises_typeerror(): + pyformat_args = {"my_object": object()} + sql = "SELECT {my_object}" + + with pytest.raises(TypeError, match="my_object has unsupported type: "): + pyformat.pyformat(sql, pyformat_args=pyformat_args) + + +def test_pyformat_with_missing_variable_raises_keyerror(): + pyformat_args: Dict[str, Any] = {} + sql = "SELECT {my_object}" + + with pytest.raises(KeyError, match="my_object"): + pyformat.pyformat(sql, pyformat_args=pyformat_args) + + +def test_pyformat_with_no_variables(): + pyformat_args: Dict[str, Any] = {} + sql = "SELECT '{{escaped curly brackets}}'" + expected_sql = "SELECT '{escaped curly brackets}'" + got_sql = pyformat.pyformat(sql, pyformat_args=pyformat_args) + assert got_sql == expected_sql + + +def test_pyformat_with_query_string_replaces_variables(): + pyformat_args = { + "my_string": "some string value", + "max_value": 2.25, + "year": 2025, + "null_value": None, + # Unreferenced values of unsupported type shouldn't cause issues. + "my_object": object(), + } + + sql = """ + SELECT {year} - year AS age, + @myparam AS myparam, + '{{my_string}}' AS escaped_string, + {my_string} AS my_string, + {null_value} AS null_value, + FROM my_dataset.my_table + WHERE height < {max_value} + """.strip() + + expected_sql = """ + SELECT 2025 - year AS age, + @myparam AS myparam, + '{my_string}' AS escaped_string, + 'some string value' AS my_string, + NULL AS null_value, + FROM my_dataset.my_table + WHERE height < 2.25 + """.strip() + + got_sql = pyformat.pyformat(sql, pyformat_args=pyformat_args) + assert got_sql == expected_sql + + +@pytest.mark.parametrize( + ("table", "expected_sql"), + ( + ( + google.cloud.bigquery.Table("my-project.my_dataset.my_table"), + "SELECT * FROM `my-project`.`my_dataset`.`my_table`", + ), + ( + google.cloud.bigquery.TableReference( + google.cloud.bigquery.DatasetReference("some-project", "some_dataset"), + "some_table", + ), + "SELECT * FROM `some-project`.`some_dataset`.`some_table`", + ), + ( + google.cloud.bigquery.table.TableListItem( + { + "tableReference": { + "projectId": "ListedProject", + "datasetId": "ListedDataset", + "tableId": "ListedTable", + } + } + ), + "SELECT * FROM `ListedProject`.`ListedDataset`.`ListedTable`", + ), + ), +) +def test_pyformat_with_table_replaces_variables(table, expected_sql): + pyformat_args = { + "table": table, + # Unreferenced values of unsupported type shouldn't cause issues. + "my_object": object(), + } + sql = "SELECT * FROM {table}" + got_sql = pyformat.pyformat(sql, pyformat_args=pyformat_args) + assert got_sql == expected_sql From 6b1fe325c4f709e2b17e6c16ef8543ed716e5067 Mon Sep 17 00:00:00 2001 From: Huan Chen <142538604+Genesis929@users.noreply.github.com> Date: Mon, 12 May 2025 11:10:55 -0700 Subject: [PATCH 30/36] Revert "chore: temporary fix for q11 (#1587)" (#1678) This reverts commit 4f7479879e2c9a890ce0aa280503e3cd4f62d9b1. --- third_party/bigframes_vendored/tpch/queries/q11.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/third_party/bigframes_vendored/tpch/queries/q11.py b/third_party/bigframes_vendored/tpch/queries/q11.py index 9d868f3343..365aa12eb9 100644 --- a/third_party/bigframes_vendored/tpch/queries/q11.py +++ b/third_party/bigframes_vendored/tpch/queries/q11.py @@ -43,4 +43,4 @@ def q(project_id: str, dataset_id: str, session: bigframes.Session): result_df = result_df.sort_values(by="VALUE", ascending=False) - result_df.to_pandas() + next(result_df.to_pandas_batches(max_results=1500)) From ab24b6a9647683cefea16a4298019e214894dc4a Mon Sep 17 00:00:00 2001 From: Shuowei Li Date: Mon, 12 May 2025 11:50:22 -0700 Subject: [PATCH 31/36] test: add gemini test who are ready in production code (#1718) --- bigframes/ml/llm.py | 1 + tests/system/load/test_llm.py | 3 ++- tests/system/small/ml/test_multimodal_llm.py | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/bigframes/ml/llm.py b/bigframes/ml/llm.py index f93ba8b720..ef74c4ac55 100644 --- a/bigframes/ml/llm.py +++ b/bigframes/ml/llm.py @@ -83,6 +83,7 @@ _GEMINI_1P5_FLASH_002_ENDPOINT, _GEMINI_2_FLASH_EXP_ENDPOINT, _GEMINI_2_FLASH_001_ENDPOINT, + _GEMINI_2_FLASH_LITE_001_ENDPOINT, ) _CLAUDE_3_SONNET_ENDPOINT = "claude-3-sonnet" diff --git a/tests/system/load/test_llm.py b/tests/system/load/test_llm.py index abb199b8ab..d346d109be 100644 --- a/tests/system/load/test_llm.py +++ b/tests/system/load/test_llm.py @@ -25,7 +25,7 @@ def llm_remote_text_pandas_df(): return pd.DataFrame( { "prompt": [ - "Please do sentiment analysis on the following text and only output a number from 0 to 5where 0 means sadness, 1 means joy, 2 means love, 3 means anger, 4 means fear, and 5 means surprise. Text: i feel beautifully emotional knowing that these women of whom i knew just a handful were holding me and my baba on our journey", + "Please do sentiment analysis on the following text and only output a number from 0 to 5 where 0 means sadness, 1 means joy, 2 means love, 3 means anger, 4 means fear, and 5 means surprise. Text: i feel beautifully emotional knowing that these women of whom i knew just a handful were holding me and my baba on our journey", "Please do sentiment analysis on the following text and only output a number from 0 to 5 where 0 means sadness, 1 means joy, 2 means love, 3 means anger, 4 means fear, and 5 means surprise. Text: i was feeling a little vain when i did this one", "Please do sentiment analysis on the following text and only output a number from 0 to 5 where 0 means sadness, 1 means joy, 2 means love, 3 means anger, 4 means fear, and 5 means surprise. Text: a father of children killed in an accident", ], @@ -43,6 +43,7 @@ def llm_remote_text_df(session, llm_remote_text_pandas_df): ( "gemini-1.5-pro-002", "gemini-1.5-flash-002", + "gemini-2.0-flash-001", ), ) def test_llm_gemini_configure_fit( diff --git a/tests/system/small/ml/test_multimodal_llm.py b/tests/system/small/ml/test_multimodal_llm.py index 19ec3d7e14..e066d00cf5 100644 --- a/tests/system/small/ml/test_multimodal_llm.py +++ b/tests/system/small/ml/test_multimodal_llm.py @@ -47,6 +47,7 @@ def test_multimodal_embedding_generator_predict_default_params_success( "gemini-1.5-flash-002", "gemini-2.0-flash-exp", "gemini-2.0-flash-001", + "gemini-2.0-flash-lite-001", ), ) @pytest.mark.flaky(retries=2) From 18780b48a17dba2b3b3542500f027ae9527f6bee Mon Sep 17 00:00:00 2001 From: Huan Chen <142538604+Genesis929@users.noreply.github.com> Date: Mon, 12 May 2025 12:12:55 -0700 Subject: [PATCH 32/36] docs: deprecate `bpd.options.bigquery.allow_large_results` in favor of `bpd.options.compute.allow_large_results` (#1597) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * update allow_large_result warning * Update bigframes/_config/bigquery_options.py Co-authored-by: Tim Sweña (Swast) * Update bigframes/_config/bigquery_options.py Co-authored-by: Tim Sweña (Swast) * Update bigframes/_config/bigquery_options.py Co-authored-by: Tim Sweña (Swast) * update setting * Update bigframes/_config/bigquery_options.py Co-authored-by: Tim Sweña (Swast) * update docs * Update bigframes/_config/bigquery_options.py * fix execute --------- Co-authored-by: Tim Sweña (Swast) --- bigframes/_config/__init__.py | 18 ++++++++++++++ bigframes/_config/bigquery_options.py | 29 +++++++++++++++------- bigframes/_config/compute_options.py | 10 +++++++- bigframes/session/bq_caching_executor.py | 13 +++++++--- tests/system/conftest.py | 12 +++------ tests/system/large/test_dataframe_io.py | 2 +- tests/system/small/test_dataframe.py | 2 +- tests/system/small/test_dataframe_io.py | 4 +-- tests/system/small/test_index_io.py | 4 +-- tests/system/small/test_progress_bar.py | 2 +- tests/system/small/test_series.py | 2 +- tests/system/small/test_series_io.py | 2 +- tests/unit/_config/test_compute_options.py | 22 ++++++++++++++++ 13 files changed, 91 insertions(+), 31 deletions(-) create mode 100644 tests/unit/_config/test_compute_options.py diff --git a/bigframes/_config/__init__.py b/bigframes/_config/__init__.py index db860e6b1d..775ef70bc7 100644 --- a/bigframes/_config/__init__.py +++ b/bigframes/_config/__init__.py @@ -150,6 +150,24 @@ def is_bigquery_thread_local(self) -> bool: """ return self._local.bigquery_options is not None + @property + def _allow_large_results(self) -> bool: + """The effective 'allow_large_results' setting. + + This value is `self.compute.allow_large_results` if set (not `None`), + otherwise it defaults to `self.bigquery.allow_large_results`. + + Returns: + bool: + Whether large query results are permitted. + - `True`: The BigQuery result size limit (e.g., 10 GB) is removed. + - `False`: Results are restricted to this limit (potentially faster). + BigQuery will raise an error if this limit is exceeded. + """ + if self.compute.allow_large_results is None: + return self.bigquery.allow_large_results + return self.compute.allow_large_results + options = Options() """Global options for default session.""" diff --git a/bigframes/_config/bigquery_options.py b/bigframes/_config/bigquery_options.py index 5155b09063..3a6008eaa8 100644 --- a/bigframes/_config/bigquery_options.py +++ b/bigframes/_config/bigquery_options.py @@ -19,10 +19,8 @@ from typing import Literal, Optional import warnings -import google.api_core.exceptions import google.auth.credentials -import bigframes.constants import bigframes.enums import bigframes.exceptions as bfe @@ -239,21 +237,34 @@ def skip_bq_connection_check(self, value: bool): @property def allow_large_results(self) -> bool: """ - Sets the flag to allow or disallow query results larger than 10 GB. + DEPRECATED: Checks the legacy global setting for allowing large results. + Use ``bpd.options.compute.allow_large_results`` instead. - The default setting for this flag is True, which allows queries to return results - exceeding 10 GB by creating an explicit destination table. If set to False, it - restricts the result size to 10 GB, and BigQuery will raise an error if this limit - is exceeded. + Warning: Accessing ``bpd.options.bigquery.allow_large_results`` is deprecated + and this property will be removed in a future version. The configuration for + handling large results has moved. Returns: - bool: True if large results are allowed with an explicit destination table, - False if results are limited to 10 GB and errors are raised when exceeded. + bool: The value of the deprecated setting. """ return self._allow_large_results @allow_large_results.setter def allow_large_results(self, value: bool): + warnings.warn( + "Setting `bpd.options.bigquery.allow_large_results` is deprecated, " + "and will be removed in the future. " + "Please use `bpd.options.compute.allow_large_results = ` instead. " + "The `bpd.options.bigquery.allow_large_results` option is ignored if " + "`bpd.options.compute.allow_large_results` is set.", + FutureWarning, + stacklevel=2, + ) + if self._session_started and self._allow_large_results != value: + raise ValueError( + SESSION_STARTED_MESSAGE.format(attribute="allow_large_results") + ) + self._allow_large_results = value @property diff --git a/bigframes/_config/compute_options.py b/bigframes/_config/compute_options.py index eb287f6065..89c0dc8d6a 100644 --- a/bigframes/_config/compute_options.py +++ b/bigframes/_config/compute_options.py @@ -86,6 +86,12 @@ class ComputeOptions: ai_ops_threshold_autofail (bool): Guards against unexpected processing of large amount of rows by semantic operators. When set to True, the operation automatically fails without asking for user inputs. + + allow_large_results (bool): + Specifies whether query results can exceed 10 GB. Defaults to False. Setting this + to False (the default) restricts results to 10 GB for potentially faster execution; + BigQuery will raise an error if this limit is exceeded. Setting to True removes + this result size limit. """ maximum_bytes_billed: Optional[int] = None @@ -97,7 +103,9 @@ class ComputeOptions: semantic_ops_threshold_autofail = False ai_ops_confirmation_threshold: Optional[int] = 0 - ai_ops_threshold_autofail = False + ai_ops_threshold_autofail: bool = False + + allow_large_results: Optional[bool] = None def assign_extra_query_labels(self, **kwargs: Any) -> None: """ diff --git a/bigframes/session/bq_caching_executor.py b/bigframes/session/bq_caching_executor.py index 72f2dfa4b5..9288fdf641 100644 --- a/bigframes/session/bq_caching_executor.py +++ b/bigframes/session/bq_caching_executor.py @@ -61,7 +61,7 @@ def with_require_table(self, value: bool) -> OutputSpec: def _get_default_output_spec() -> OutputSpec: return OutputSpec( - require_bq_table=bigframes.options.bigquery.allow_large_results, cluster_cols=() + require_bq_table=bigframes.options._allow_large_results, cluster_cols=() ) @@ -157,9 +157,6 @@ def execute( ordered: bool = True, use_explicit_destination: Optional[bool] = None, ) -> executor.ExecuteResult: - if use_explicit_destination is None: - use_explicit_destination = bigframes.options.bigquery.allow_large_results - if bigframes.options.compute.enable_multi_query_execution: self._simplify_with_caching(array_value) @@ -553,6 +550,14 @@ def _execute_plan( else: size_bytes = None + if size_bytes is not None and size_bytes >= MAX_SMALL_RESULT_BYTES: + msg = bfe.format_message( + "The query result size has exceeded 10 GB. In BigFrames 2.0 and " + "later, you might need to manually set `allow_large_results=True` in " + "the IO method or adjust the BigFrames option: " + "`bigframes.options.compute.allow_large_results=True`." + ) + warnings.warn(msg, FutureWarning) # Runs strict validations to ensure internal type predictions and ibis are completely in sync # Do not execute these validations outside of testing suite. if "PYTEST_CURRENT_TEST" in os.environ: diff --git a/tests/system/conftest.py b/tests/system/conftest.py index 458e8e0ad2..fb7d8d4e32 100644 --- a/tests/system/conftest.py +++ b/tests/system/conftest.py @@ -142,7 +142,7 @@ def resourcemanager_client( @pytest.fixture(scope="session") def session() -> Generator[bigframes.Session, None, None]: - context = bigframes.BigQueryOptions(location="US", allow_large_results=False) + context = bigframes.BigQueryOptions(location="US") session = bigframes.Session(context=context) yield session session.close() # close generated session at cleanup time @@ -158,9 +158,7 @@ def session_load() -> Generator[bigframes.Session, None, None]: @pytest.fixture(scope="session", params=["strict", "partial"]) def maybe_ordered_session(request) -> Generator[bigframes.Session, None, None]: - context = bigframes.BigQueryOptions( - location="US", ordering_mode=request.param, allow_large_results=False - ) + context = bigframes.BigQueryOptions(location="US", ordering_mode=request.param) session = bigframes.Session(context=context) yield session session.close() # close generated session at cleanup type @@ -168,9 +166,7 @@ def maybe_ordered_session(request) -> Generator[bigframes.Session, None, None]: @pytest.fixture(scope="session") def unordered_session() -> Generator[bigframes.Session, None, None]: - context = bigframes.BigQueryOptions( - location="US", ordering_mode="partial", allow_large_results=False - ) + context = bigframes.BigQueryOptions(location="US", ordering_mode="partial") session = bigframes.Session(context=context) yield session session.close() # close generated session at cleanup type @@ -1419,7 +1415,7 @@ def floats_product_bf(session, floats_product_pd): @pytest.fixture(scope="session", autouse=True) def use_fast_query_path(): - with bpd.option_context("bigquery.allow_large_results", False): + with bpd.option_context("compute.allow_large_results", False): yield diff --git a/tests/system/large/test_dataframe_io.py b/tests/system/large/test_dataframe_io.py index b90139b12f..b10e361129 100644 --- a/tests/system/large/test_dataframe_io.py +++ b/tests/system/large/test_dataframe_io.py @@ -18,7 +18,7 @@ import bigframes WIKIPEDIA_TABLE = "bigquery-public-data.samples.wikipedia" -LARGE_TABLE_OPTION = "bigquery.allow_large_results" +LARGE_TABLE_OPTION = "compute.allow_large_results" def test_to_pandas_batches_raise_when_large_result_not_allowed(session): diff --git a/tests/system/small/test_dataframe.py b/tests/system/small/test_dataframe.py index ce291d4999..00c11d073e 100644 --- a/tests/system/small/test_dataframe.py +++ b/tests/system/small/test_dataframe.py @@ -5081,7 +5081,7 @@ def test_df_bool_interpretation_error(scalars_df_index): def test_query_job_setters(scalars_df_default_index: dataframe.DataFrame): # if allow_large_results=False, might not create query job - with bigframes.option_context("bigquery.allow_large_results", True): + with bigframes.option_context("compute.allow_large_results", True): job_ids = set() repr(scalars_df_default_index) assert scalars_df_default_index.query_job is not None diff --git a/tests/system/small/test_dataframe_io.py b/tests/system/small/test_dataframe_io.py index e210fed522..d24b592b0d 100644 --- a/tests/system/small/test_dataframe_io.py +++ b/tests/system/small/test_dataframe_io.py @@ -254,7 +254,7 @@ def test_to_pandas_array_struct_correct_result(session): def test_to_pandas_override_global_option(scalars_df_index): # Direct call to_pandas uses global default setting (allow_large_results=True), # table has 'bqdf' prefix. - with bigframes.option_context("bigquery.allow_large_results", True): + with bigframes.option_context("compute.allow_large_results", True): scalars_df_index.to_pandas() table_id = scalars_df_index._query_job.destination.table_id @@ -324,7 +324,7 @@ def test_to_pandas_dry_run(session, scalars_pandas_df_multi_index): def test_to_arrow_override_global_option(scalars_df_index): # Direct call to_arrow uses global default setting (allow_large_results=True), - with bigframes.option_context("bigquery.allow_large_results", True): + with bigframes.option_context("compute.allow_large_results", True): scalars_df_index.to_arrow() table_id = scalars_df_index._query_job.destination.table_id diff --git a/tests/system/small/test_index_io.py b/tests/system/small/test_index_io.py index fcb3fa3920..78e561c2fd 100644 --- a/tests/system/small/test_index_io.py +++ b/tests/system/small/test_index_io.py @@ -15,7 +15,7 @@ def test_to_pandas_override_global_option(scalars_df_index): - with bigframes.option_context("bigquery.allow_large_results", True): + with bigframes.option_context("compute.allow_large_results", True): bf_index = scalars_df_index.index @@ -39,7 +39,7 @@ def test_to_pandas_dry_run(scalars_df_index): def test_to_numpy_override_global_option(scalars_df_index): - with bigframes.option_context("bigquery.allow_large_results", True): + with bigframes.option_context("compute.allow_large_results", True): bf_index = scalars_df_index.index diff --git a/tests/system/small/test_progress_bar.py b/tests/system/small/test_progress_bar.py index 9c61c8ea5b..35b540966e 100644 --- a/tests/system/small/test_progress_bar.py +++ b/tests/system/small/test_progress_bar.py @@ -64,7 +64,7 @@ def test_progress_bar_scalar_allow_large_results( capsys.readouterr() # clear output with bf.option_context( - "display.progress_bar", "terminal", "bigquery.allow_large_results", "True" + "display.progress_bar", "terminal", "compute.allow_large_results", "True" ): penguins_df_default_index["body_mass_g"].head(10).mean() diff --git a/tests/system/small/test_series.py b/tests/system/small/test_series.py index 3852c417fa..99526a65d2 100644 --- a/tests/system/small/test_series.py +++ b/tests/system/small/test_series.py @@ -3960,7 +3960,7 @@ def test_series_bool_interpretation_error(scalars_df_index): def test_query_job_setters(scalars_dfs): # if allow_large_results=False, might not create query job - with bigframes.option_context("bigquery.allow_large_results", True): + with bigframes.option_context("compute.allow_large_results", True): job_ids = set() df, _ = scalars_dfs series = df["int64_col"] diff --git a/tests/system/small/test_series_io.py b/tests/system/small/test_series_io.py index 235ae65750..5390d65268 100644 --- a/tests/system/small/test_series_io.py +++ b/tests/system/small/test_series_io.py @@ -19,7 +19,7 @@ def test_to_pandas_override_global_option(scalars_df_index): - with bigframes.option_context("bigquery.allow_large_results", True): + with bigframes.option_context("compute.allow_large_results", True): bf_series = scalars_df_index["int64_col"] diff --git a/tests/unit/_config/test_compute_options.py b/tests/unit/_config/test_compute_options.py new file mode 100644 index 0000000000..e06eb76c37 --- /dev/null +++ b/tests/unit/_config/test_compute_options.py @@ -0,0 +1,22 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +import bigframes._config as config + + +def test_default_options(): + options = config.compute_options.ComputeOptions() + + assert options.allow_large_results is None + assert config.options._allow_large_results is False From 047976315dcbaed86e50d47f545b76c3a513dafb Mon Sep 17 00:00:00 2001 From: Shenyang Cai Date: Mon, 12 May 2025 13:25:04 -0700 Subject: [PATCH 33/36] feat: add `isocalendar()` for dt accessor" (#1717) --- bigframes/core/compile/scalar_op_compiler.py | 16 ++++++++++ bigframes/operations/__init__.py | 10 ++++-- bigframes/operations/date_ops.py | 14 ++++++++ bigframes/operations/datetimes.py | 13 ++++++-- .../system/small/operations/test_datetimes.py | 15 +++++++++ .../pandas/core/indexes/accessor.py | 32 +++++++++++++++++++ 6 files changed, 96 insertions(+), 4 deletions(-) diff --git a/bigframes/core/compile/scalar_op_compiler.py b/bigframes/core/compile/scalar_op_compiler.py index e2dfa38ce1..78e373121e 100644 --- a/bigframes/core/compile/scalar_op_compiler.py +++ b/bigframes/core/compile/scalar_op_compiler.py @@ -667,6 +667,22 @@ def date_op_impl(x: ibis_types.Value): return typing.cast(ibis_types.TimestampValue, x).date() +@scalar_op_compiler.register_unary_op(ops.iso_day_op) +def iso_day_op_impl(x: ibis_types.Value): + # Plus 1 because iso day of week uses 1-based indexing + return dayofweek_op_impl(x) + 1 + + +@scalar_op_compiler.register_unary_op(ops.iso_week_op) +def iso_week_op_impl(x: ibis_types.Value): + return typing.cast(ibis_types.TimestampValue, x).week_of_year() + + +@scalar_op_compiler.register_unary_op(ops.iso_year_op) +def iso_year_op_impl(x: ibis_types.Value): + return typing.cast(ibis_types.TimestampValue, x).iso_year() + + @scalar_op_compiler.register_unary_op(ops.dayofweek_op) def dayofweek_op_impl(x: ibis_types.Value): return ( diff --git a/bigframes/operations/__init__.py b/bigframes/operations/__init__.py index 0f9b64b760..e5da674a8c 100644 --- a/bigframes/operations/__init__.py +++ b/bigframes/operations/__init__.py @@ -43,6 +43,9 @@ day_op, dayofweek_op, dayofyear_op, + iso_day_op, + iso_week_op, + iso_year_op, month_op, quarter_op, year_op, @@ -260,11 +263,14 @@ # Date ops "date_diff_op", "day_op", - "month_op", - "year_op", "dayofweek_op", "dayofyear_op", + "iso_day_op", + "iso_week_op", + "iso_year_op", + "month_op", "quarter_op", + "year_op", # Time ops "hour_op", "minute_op", diff --git a/bigframes/operations/date_ops.py b/bigframes/operations/date_ops.py index 9bcdddb8df..0b91c86b11 100644 --- a/bigframes/operations/date_ops.py +++ b/bigframes/operations/date_ops.py @@ -34,6 +34,20 @@ type_signature=op_typing.DATELIKE_ACCESSOR, ) +iso_day_op = base_ops.create_unary_op( + name="iso_day", type_signature=op_typing.DATELIKE_ACCESSOR +) + +iso_week_op = base_ops.create_unary_op( + name="iso_weeek", + type_signature=op_typing.DATELIKE_ACCESSOR, +) + +iso_year_op = base_ops.create_unary_op( + name="iso_year", + type_signature=op_typing.DATELIKE_ACCESSOR, +) + dayofweek_op = base_ops.create_unary_op( name="dayofweek", type_signature=op_typing.DATELIKE_ACCESSOR, diff --git a/bigframes/operations/datetimes.py b/bigframes/operations/datetimes.py index e2439f6393..56320e7cc6 100644 --- a/bigframes/operations/datetimes.py +++ b/bigframes/operations/datetimes.py @@ -21,11 +21,11 @@ import bigframes_vendored.pandas.core.indexes.accessor as vendordt import pandas -from bigframes import dtypes +from bigframes import dataframe, dtypes, series from bigframes.core import log_adapter +from bigframes.core.reshape import concat import bigframes.operations as ops import bigframes.operations.base -import bigframes.series as series _ONE_DAY = pandas.Timedelta("1d") _ONE_SECOND = pandas.Timedelta("1s") @@ -69,6 +69,15 @@ def year(self) -> series.Series: def month(self) -> series.Series: return self._apply_unary_op(ops.month_op) + def isocalendar(self) -> dataframe.DataFrame: + years = self._apply_unary_op(ops.iso_year_op) + weeks = self._apply_unary_op(ops.iso_week_op) + days = self._apply_unary_op(ops.iso_day_op) + + result = concat.concat([years, weeks, days], axis=1) + result.columns = pandas.Index(["year", "week", "day"]) + return result + # Time accessors @property def hour(self) -> series.Series: diff --git a/tests/system/small/operations/test_datetimes.py b/tests/system/small/operations/test_datetimes.py index 69f68ca5de..bbecf40e0b 100644 --- a/tests/system/small/operations/test_datetimes.py +++ b/tests/system/small/operations/test_datetimes.py @@ -229,6 +229,21 @@ def test_dt_year(scalars_dfs, col_name): ) +def test_dt_isocalendar(session): + # We don't re-use the exisintg scalars_dfs fixture because iso calendar + # get tricky when a new year starts, but the dataset `scalars_dfs` does not cover + # this case. + pd_s = pd.Series(pd.date_range("2009-12-25", "2010-01-07", freq="d")) + bf_s = session.read_pandas(pd_s) + + actual_result = bf_s.dt.isocalendar().to_pandas() + + expected_result = pd_s.dt.isocalendar() + testing.assert_frame_equal( + actual_result, expected_result, check_dtype=False, check_index_type=False + ) + + @pytest.mark.parametrize( ("col_name",), DATETIME_COL_NAMES, diff --git a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py index e642d11ca4..469f35f181 100644 --- a/third_party/bigframes_vendored/pandas/core/indexes/accessor.py +++ b/third_party/bigframes_vendored/pandas/core/indexes/accessor.py @@ -199,6 +199,38 @@ def month(self): raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + def isocalendar(self): + """ + Calculate year, week, and day according to the ISO 8601 standard. + + **Examples:** + >>> import pandas as pd + >>> import bigframes.pandas as bpd + >>> bpd.options.display.progress_bar = None + >>> s = bpd.Series( + ... pd.date_range('2009-12-27', '2010-01-04', freq='d').to_series() + ... ) + >>> s.dt.isocalendar() + year week day + 2009-12-27 00:00:00 2009 52 7 + 2009-12-28 00:00:00 2009 53 1 + 2009-12-29 00:00:00 2009 53 2 + 2009-12-30 00:00:00 2009 53 3 + 2009-12-31 00:00:00 2009 53 4 + 2010-01-01 00:00:00 2009 53 5 + 2010-01-02 00:00:00 2009 53 6 + 2010-01-03 00:00:00 2009 53 7 + 2010-01-04 00:00:00 2010 1 1 + + [9 rows x 3 columns] + + + Returns: DataFrame + With columns year, week and day. + + + """ + @property def second(self): """The seconds of the datetime. From c3c830cf20397830d531a89edf5302aede5d48a0 Mon Sep 17 00:00:00 2001 From: Shenyang Cai Date: Mon, 12 May 2025 13:44:47 -0700 Subject: [PATCH 34/36] feat: add `DatetimeIndex` class (#1719) * [WIP] added DatetimeIndex. Docs and tests to come next * add tests and docs * fix mypy * fix lint * handle empty index scenario --- bigframes/core/indexes/__init__.py | 2 + bigframes/core/indexes/base.py | 21 +++- bigframes/core/indexes/datetimes.py | 56 +++++++++ bigframes/pandas/__init__.py | 2 + docs/reference/bigframes.pandas/indexing.rst | 12 ++ tests/system/small/core/indexes/__init__.py | 13 +++ .../small/core/indexes/test_datetimes.py | 46 ++++++++ .../pandas/core/indexes/datetimes.py | 106 ++++++++++++++++++ 8 files changed, 254 insertions(+), 4 deletions(-) create mode 100644 bigframes/core/indexes/datetimes.py create mode 100644 tests/system/small/core/indexes/__init__.py create mode 100644 tests/system/small/core/indexes/test_datetimes.py create mode 100644 third_party/bigframes_vendored/pandas/core/indexes/datetimes.py diff --git a/bigframes/core/indexes/__init__.py b/bigframes/core/indexes/__init__.py index 0a95adcd83..dfe361aa76 100644 --- a/bigframes/core/indexes/__init__.py +++ b/bigframes/core/indexes/__init__.py @@ -13,9 +13,11 @@ # limitations under the License. from bigframes.core.indexes.base import Index +from bigframes.core.indexes.datetimes import DatetimeIndex from bigframes.core.indexes.multi import MultiIndex __all__ = [ "Index", "MultiIndex", + "DatetimeIndex", ] diff --git a/bigframes/core/indexes/base.py b/bigframes/core/indexes/base.py index eac1f58eae..9b4b5e4290 100644 --- a/bigframes/core/indexes/base.py +++ b/bigframes/core/indexes/base.py @@ -25,6 +25,7 @@ import numpy as np import pandas +from bigframes import dtypes import bigframes.core.block_transforms as block_ops import bigframes.core.blocks as blocks import bigframes.core.expression as ex @@ -90,12 +91,17 @@ def __new__( block = df.DataFrame(pd_df, session=session)._block # TODO: Support more index subtypes - from bigframes.core.indexes.multi import MultiIndex - if len(block._index_columns) <= 1: - klass = cls + if len(block._index_columns) > 1: + from bigframes.core.indexes.multi import MultiIndex + + klass: type[Index] = MultiIndex # type hint to make mypy happy + elif _should_create_datetime_index(block): + from bigframes.core.indexes.datetimes import DatetimeIndex + + klass = DatetimeIndex else: - klass = MultiIndex + klass = cls result = typing.cast(Index, object.__new__(klass)) result._query_job = None @@ -555,3 +561,10 @@ def to_numpy(self, dtype=None, *, allow_large_results=None, **kwargs) -> np.ndar def __len__(self): return self.shape[0] + + +def _should_create_datetime_index(block: blocks.Block) -> bool: + if len(block.index.dtypes) != 1: + return False + + return dtypes.is_datetime_like(block.index.dtypes[0]) diff --git a/bigframes/core/indexes/datetimes.py b/bigframes/core/indexes/datetimes.py new file mode 100644 index 0000000000..23ad8b03b4 --- /dev/null +++ b/bigframes/core/indexes/datetimes.py @@ -0,0 +1,56 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + +"""An index based on a single column with a datetime-like data type.""" + +from __future__ import annotations + +from bigframes_vendored.pandas.core.indexes import ( + datetimes as vendored_pandas_datetime_index, +) + +from bigframes.core import expression as ex +from bigframes.core.indexes.base import Index +from bigframes.operations import date_ops + + +class DatetimeIndex(Index, vendored_pandas_datetime_index.DatetimeIndex): + __doc__ = vendored_pandas_datetime_index.DatetimeIndex.__doc__ + + # Must be above 5000 for pandas to delegate to bigframes for binops + __pandas_priority__ = 12000 + + @property + def year(self) -> Index: + return self._apply_unary_expr(date_ops.year_op.as_expr(ex.free_var("arg"))) + + @property + def month(self) -> Index: + return self._apply_unary_expr(date_ops.month_op.as_expr(ex.free_var("arg"))) + + @property + def day(self) -> Index: + return self._apply_unary_expr(date_ops.day_op.as_expr(ex.free_var("arg"))) + + @property + def dayofweek(self) -> Index: + return self._apply_unary_expr(date_ops.dayofweek_op.as_expr(ex.free_var("arg"))) + + @property + def day_of_week(self) -> Index: + return self.dayofweek + + @property + def weekday(self) -> Index: + return self.dayofweek diff --git a/bigframes/pandas/__init__.py b/bigframes/pandas/__init__.py index 8e1e03e024..e90f123778 100644 --- a/bigframes/pandas/__init__.py +++ b/bigframes/pandas/__init__.py @@ -271,6 +271,7 @@ def clean_up_by_session_id( DataFrame = bigframes.dataframe.DataFrame Index = bigframes.core.indexes.Index MultiIndex = bigframes.core.indexes.MultiIndex +DatetimeIndex = bigframes.core.indexes.DatetimeIndex Series = bigframes.series.Series __version__ = bigframes.version.__version__ @@ -357,6 +358,7 @@ def reset_session(): "DataFrame", "Index", "MultiIndex", + "DatetimeIndex", "Series", "__version__", # Other public pandas attributes diff --git a/docs/reference/bigframes.pandas/indexing.rst b/docs/reference/bigframes.pandas/indexing.rst index 2cc1acfabf..e25e8652ec 100644 --- a/docs/reference/bigframes.pandas/indexing.rst +++ b/docs/reference/bigframes.pandas/indexing.rst @@ -7,3 +7,15 @@ Index objects :members: :inherited-members: :undoc-members: + + +.. autoclass:: bigframes.core.indexes.multi.MultiIndex + :members: + :inherited-members: + :undoc-members: + + +.. autoclass:: bigframes.core.indexes.datetimes.DatetimeIndex + :members: + :inherited-members: + :undoc-members: \ No newline at end of file diff --git a/tests/system/small/core/indexes/__init__.py b/tests/system/small/core/indexes/__init__.py new file mode 100644 index 0000000000..0a2669d7a2 --- /dev/null +++ b/tests/system/small/core/indexes/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. diff --git a/tests/system/small/core/indexes/test_datetimes.py b/tests/system/small/core/indexes/test_datetimes.py new file mode 100644 index 0000000000..40ce310b31 --- /dev/null +++ b/tests/system/small/core/indexes/test_datetimes.py @@ -0,0 +1,46 @@ +# Copyright 2025 Google LLC +# +# Licensed 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. + + +import pandas +import pandas.testing +import pytest + + +@pytest.fixture(scope="module") +def datetime_indexes(session): + pd_index = pandas.date_range("2024-12-25", periods=10, freq="d") + bf_index = session.read_pandas(pd_index) + + return bf_index, pd_index + + +@pytest.mark.parametrize( + "access", + [ + pytest.param(lambda x: x.year, id="year"), + pytest.param(lambda x: x.month, id="month"), + pytest.param(lambda x: x.day, id="day"), + pytest.param(lambda x: x.dayofweek, id="dayofweek"), + pytest.param(lambda x: x.day_of_week, id="day_of_week"), + pytest.param(lambda x: x.weekday, id="weekday"), + ], +) +def test_datetime_index_properties(datetime_indexes, access): + bf_index, pd_index = datetime_indexes + + actual_result = access(bf_index).to_pandas() + + expected_result = access(pd_index).astype(pandas.Int64Dtype()) + pandas.testing.assert_index_equal(actual_result, expected_result) diff --git a/third_party/bigframes_vendored/pandas/core/indexes/datetimes.py b/third_party/bigframes_vendored/pandas/core/indexes/datetimes.py new file mode 100644 index 0000000000..105a376728 --- /dev/null +++ b/third_party/bigframes_vendored/pandas/core/indexes/datetimes.py @@ -0,0 +1,106 @@ +# Contains code from https://github.com/pandas-dev/pandas/blob/main/pandas/core/indexes/datetimes.py + +from __future__ import annotations + +from bigframes_vendored import constants +from bigframes_vendored.pandas.core.indexes import base + + +class DatetimeIndex(base.Index): + """Immutable sequence used for indexing and alignment with datetime-like values""" + + @property + def year(self) -> base.Index: + """The year of the datetime + + **Examples:** + + >>> import bigframes.pandas as bpd + >>> import pandas as pd + >>> bpd.options.display.progress_bar = None + + >>> idx = bpd.Index([pd.Timestamp("20250215")]) + >>> idx.year + Index([2025], dtype='Int64') + """ + raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + + @property + def month(self) -> base.Index: + """The month as January=1, December=12. + + **Examples:** + + >>> import bigframes.pandas as bpd + >>> import pandas as pd + >>> bpd.options.display.progress_bar = None + + >>> idx = bpd.Index([pd.Timestamp("20250215")]) + >>> idx.month + Index([2], dtype='Int64') + """ + raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + + @property + def day(self) -> base.Index: + """The day of the datetime. + + **Examples:** + + >>> import bigframes.pandas as bpd + >>> import pandas as pd + >>> bpd.options.display.progress_bar = None + + >>> idx = bpd.Index([pd.Timestamp("20250215")]) + >>> idx.day + Index([15], dtype='Int64') + """ + raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + + @property + def day_of_week(self) -> base.Index: + """The day of the week with Monday=0, Sunday=6. + + **Examples:** + + >>> import bigframes.pandas as bpd + >>> import pandas as pd + >>> bpd.options.display.progress_bar = None + + >>> idx = bpd.Index([pd.Timestamp("20250215")]) + >>> idx.day_of_week + Index([5], dtype='Int64') + """ + raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + + @property + def dayofweek(self) -> base.Index: + """The day of the week with Monday=0, Sunday=6. + + **Examples:** + + >>> import bigframes.pandas as bpd + >>> import pandas as pd + >>> bpd.options.display.progress_bar = None + + >>> idx = bpd.Index([pd.Timestamp("20250215")]) + >>> idx.dayofweek + Index([5], dtype='Int64') + """ + raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) + + @property + def weekday(self) -> base.Index: + """The day of the week with Monday=0, Sunday=6. + + **Examples:** + + >>> import bigframes.pandas as bpd + >>> import pandas as pd + >>> bpd.options.display.progress_bar = None + + >>> idx = bpd.Index([pd.Timestamp("20250215")]) + >>> idx.weekday + Index([5], dtype='Int64') + """ + raise NotImplementedError(constants.ABSTRACT_METHOD_ERROR_MESSAGE) From 6629e6664079e0fd503aae67801d217d3c35d069 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tim=20Swe=C3=B1a=20=28Swast=29?= Date: Mon, 12 May 2025 16:31:01 -0500 Subject: [PATCH 35/36] chore: add dry_run parameter to _read_gbq_colab (#1721) --- bigframes/session/__init__.py | 30 ++++++++++++---- bigframes/session/dry_runs.py | 22 +++++++----- bigframes/testing/mocks.py | 10 ++++-- .../small/session/test_read_gbq_colab.py | 36 ++----------------- tests/unit/session/test_read_gbq_colab.py | 36 +++++++++++++++++++ 5 files changed, 83 insertions(+), 51 deletions(-) diff --git a/bigframes/session/__init__.py b/bigframes/session/__init__.py index b54dc1d691..17945f0be6 100644 --- a/bigframes/session/__init__.py +++ b/bigframes/session/__init__.py @@ -477,14 +477,34 @@ def _register_object( ): self._objects.append(weakref.ref(object)) + @overload def _read_gbq_colab( self, query: str, - # TODO: Add a callback parameter that takes some kind of Event object. - # TODO: Add dry_run parameter. *, pyformat_args: Optional[Dict[str, Any]] = None, + dry_run: Literal[False] = ..., ) -> dataframe.DataFrame: + ... + + @overload + def _read_gbq_colab( + self, + query: str, + *, + pyformat_args: Optional[Dict[str, Any]] = None, + dry_run: Literal[True] = ..., + ) -> pandas.Series: + ... + + def _read_gbq_colab( + self, + query: str, + # TODO: Add a callback parameter that takes some kind of Event object. + *, + pyformat_args: Optional[Dict[str, Any]] = None, + dry_run: bool = False, + ) -> Union[dataframe.DataFrame, pandas.Series]: """A version of read_gbq that has the necessary default values for use in colab integrations. This includes, no ordering, no index, no progress bar, always use string @@ -501,16 +521,13 @@ def _read_gbq_colab( None, this function always assumes {var} refers to a variable that is supposed to be supplied in this dictionary. """ - # TODO: Allow for a table ID to avoid queries like with read_gbq? - if pyformat_args is None: pyformat_args = {} - # TODO: move this to read_gbq_query if/when we expose this feature - # beyond in _read_gbq_colab. query = bigframes.core.pyformat.pyformat( query, pyformat_args=pyformat_args, + # TODO: add dry_run parameter to avoid API calls for data in pyformat_args ) return self._loader.read_gbq_query( @@ -518,6 +535,7 @@ def _read_gbq_colab( index_col=bigframes.enums.DefaultIndexKind.NULL, api_name="read_gbq_colab", force_total_order=False, + dry_run=typing.cast(Union[Literal[False], Literal[True]], dry_run), ) @overload diff --git a/bigframes/session/dry_runs.py b/bigframes/session/dry_runs.py index 4d5b41345e..caf3dfc2bb 100644 --- a/bigframes/session/dry_runs.py +++ b/bigframes/session/dry_runs.py @@ -101,20 +101,23 @@ def get_query_stats( job_api_repr = copy.deepcopy(query_job._properties) - job_ref = job_api_repr["jobReference"] + # jobReference might not be populated for "job optional" queries. + job_ref = job_api_repr.get("jobReference", {}) for key, val in job_ref.items(): index.append(key) values.append(val) + configuration = job_api_repr.get("configuration", {}) index.append("jobType") - values.append(job_api_repr["configuration"]["jobType"]) + values.append(configuration.get("jobType", None)) - query_config = job_api_repr["configuration"]["query"] + query_config = configuration.get("query", {}) for key in ("destinationTable", "useLegacySql"): index.append(key) - values.append(query_config.get(key)) + values.append(query_config.get(key, None)) - query_stats = job_api_repr["statistics"]["query"] + statistics = job_api_repr.get("statistics", {}) + query_stats = statistics.get("query", {}) for key in ( "referencedTables", "totalBytesProcessed", @@ -122,13 +125,14 @@ def get_query_stats( "statementType", ): index.append(key) - values.append(query_stats.get(key)) + values.append(query_stats.get(key, None)) + creation_time = statistics.get("creationTime", None) index.append("creationTime") values.append( - pandas.Timestamp( - job_api_repr["statistics"]["creationTime"], unit="ms", tz="UTC" - ) + pandas.Timestamp(creation_time, unit="ms", tz="UTC") + if creation_time is not None + else None ) return pandas.Series(values, index=index) diff --git a/bigframes/testing/mocks.py b/bigframes/testing/mocks.py index d0cfb50ad5..528835f6da 100644 --- a/bigframes/testing/mocks.py +++ b/bigframes/testing/mocks.py @@ -12,6 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. +import copy import datetime from typing import Optional, Sequence import unittest.mock as mock @@ -78,11 +79,14 @@ def create_bigquery_session( type(table).num_rows = mock.PropertyMock(return_value=1000000000) bqclient.get_table.return_value = table + queries = [] job_configs = [] def query_mock(query, *args, job_config=None, **kwargs): - job_configs.append(job_config) + queries.append(query) + job_configs.append(copy.deepcopy(job_config)) query_job = mock.create_autospec(google.cloud.bigquery.QueryJob) + query_job._properties = {} type(query_job).destination = mock.PropertyMock( return_value=anonymous_dataset.table("test_table"), ) @@ -100,7 +104,8 @@ def query_mock(query, *args, job_config=None, **kwargs): existing_query_and_wait = bqclient.query_and_wait def query_and_wait_mock(query, *args, job_config=None, **kwargs): - job_configs.append(job_config) + queries.append(query) + job_configs.append(copy.deepcopy(job_config)) if query.startswith("SELECT CURRENT_TIMESTAMP()"): return iter([[datetime.datetime.now()]]) else: @@ -118,6 +123,7 @@ def query_and_wait_mock(query, *args, job_config=None, **kwargs): session._bq_connection_manager = mock.create_autospec( bigframes.clients.BqConnectionManager, instance=True ) + session._queries = queries # type: ignore session._job_configs = job_configs # type: ignore return session diff --git a/tests/system/small/session/test_read_gbq_colab.py b/tests/system/small/session/test_read_gbq_colab.py index 54fdd4014e..00ce0c722b 100644 --- a/tests/system/small/session/test_read_gbq_colab.py +++ b/tests/system/small/session/test_read_gbq_colab.py @@ -47,7 +47,7 @@ def test_read_gbq_colab_to_pandas_batches_preserves_order_by(maybe_ordered_sessi def test_read_gbq_colab_includes_formatted_scalars(session): pyformat_args = { "some_integer": 123, - "some_string": "This could be dangerous, but we esape it", + "some_string": "This could be dangerous, but we escape it", # This is not a supported type, but ignored if not referenced. "some_object": object(), } @@ -66,39 +66,7 @@ def test_read_gbq_colab_includes_formatted_scalars(session): { "some_integer": pandas.Series([123], dtype=pandas.Int64Dtype()), "some_string": pandas.Series( - ["This could be dangerous, but we esape it"], - dtype="string[pyarrow]", - ), - "escaped": pandas.Series(["{escaped}"], dtype="string[pyarrow]"), - } - ), - ) - - -def test_read_gbq_colab_includes_formatted_bigframes_dataframe(session): - pyformat_args = { - # TODO: put a bigframes DataFrame here. - "some_integer": 123, - "some_string": "This could be dangerous, but we esape it", - # This is not a supported type, but ignored if not referenced. - "some_object": object(), - } - df = session._read_gbq_colab( - """ - SELECT {some_integer} as some_integer, - {some_string} as some_string, - '{{escaped}}' as escaped - """, - pyformat_args=pyformat_args, - ) - result = df.to_pandas() - pandas.testing.assert_frame_equal( - result, - pandas.DataFrame( - { - "some_integer": pandas.Series([123], dtype=pandas.Int64Dtype()), - "some_string": pandas.Series( - ["This could be dangerous, but we esape it"], + ["This could be dangerous, but we escape it"], dtype="string[pyarrow]", ), "escaped": pandas.Series(["{escaped}"], dtype="string[pyarrow]"), diff --git a/tests/unit/session/test_read_gbq_colab.py b/tests/unit/session/test_read_gbq_colab.py index ddca220a79..a27abd5f6c 100644 --- a/tests/unit/session/test_read_gbq_colab.py +++ b/tests/unit/session/test_read_gbq_colab.py @@ -30,3 +30,39 @@ def test_read_gbq_colab_includes_label(): label_values.extend(config.labels.values()) assert "read_gbq_colab" in label_values + + +def test_read_gbq_colab_includes_formatted_values_in_dry_run(): + session = mocks.create_bigquery_session() + + pyformat_args = { + "some_integer": 123, + "some_string": "This could be dangerous, but we escape it", + # This is not a supported type, but ignored if not referenced. + "some_object": object(), + } + _ = session._read_gbq_colab( + """ + SELECT {some_integer} as some_integer, + {some_string} as some_string, + '{{escaped}}' as escaped + """, + pyformat_args=pyformat_args, + dry_run=True, + ) + expected = """ + SELECT 123 as some_integer, + 'This could be dangerous, but we escape it' as some_string, + '{escaped}' as escaped + """ + queries = session._queries # type: ignore + configs = session._job_configs # type: ignore + + for query, config in zip(queries, configs): + if config is None: + continue + if config.dry_run: + break + + assert config.dry_run + assert query.strip() == expected.strip() From 796168174392fcbb994574e7eb98ee0b96d370ce Mon Sep 17 00:00:00 2001 From: "release-please[bot]" <55107282+release-please[bot]@users.noreply.github.com> Date: Mon, 12 May 2025 14:59:03 -0700 Subject: [PATCH 36/36] chore(main): release 2.4.0 (#1695) Co-authored-by: release-please[bot] <55107282+release-please[bot]@users.noreply.github.com> --- CHANGELOG.md | 46 +++++++++++++++++++++++ bigframes/version.py | 4 +- third_party/bigframes_vendored/version.py | 4 +- 3 files changed, 50 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8cf9b3fc7d..0fafd11c8a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,52 @@ [1]: https://pypi.org/project/bigframes/#history +## [2.4.0](https://github.com/googleapis/python-bigquery-dataframes/compare/v2.3.0...v2.4.0) (2025-05-12) + + +### Features + +* Add "dayofyear" property for `dt` accessors ([#1692](https://github.com/googleapis/python-bigquery-dataframes/issues/1692)) ([9d4a59d](https://github.com/googleapis/python-bigquery-dataframes/commit/9d4a59ddf22793d4e0587ea2f8648fae937875f3)) +* Add `.dt.days`, `.dt.seconds`, `dt.microseconds`, and `dt.total_seconds()` for timedelta series. ([#1713](https://github.com/googleapis/python-bigquery-dataframes/issues/1713)) ([2b3a45f](https://github.com/googleapis/python-bigquery-dataframes/commit/2b3a45f8c1fd299ee97cf1c343df7c80175b4287)) +* Add `DatetimeIndex` class ([#1719](https://github.com/googleapis/python-bigquery-dataframes/issues/1719)) ([c3c830c](https://github.com/googleapis/python-bigquery-dataframes/commit/c3c830cf20397830d531a89edf5302aede5d48a0)) +* Add `isocalendar()` for dt accessor" ([#1717](https://github.com/googleapis/python-bigquery-dataframes/issues/1717)) ([0479763](https://github.com/googleapis/python-bigquery-dataframes/commit/047976315dcbaed86e50d47f545b76c3a513dafb)) +* Add bigframes.bigquery.json_value ([#1697](https://github.com/googleapis/python-bigquery-dataframes/issues/1697)) ([46a9c53](https://github.com/googleapis/python-bigquery-dataframes/commit/46a9c53256be2a293f96122ba6b330564383bcd5)) +* Add blob.exif function support ([#1703](https://github.com/googleapis/python-bigquery-dataframes/issues/1703)) ([3f79528](https://github.com/googleapis/python-bigquery-dataframes/commit/3f79528781abe9bfc122f6f6e26bfa08b029265a)) +* Add inplace arg support to sort methods ([#1710](https://github.com/googleapis/python-bigquery-dataframes/issues/1710)) ([d1ccb52](https://github.com/googleapis/python-bigquery-dataframes/commit/d1ccb524ea26deac1cf9e481e9d55f9ae166247b)) +* Improve error message in `Series.apply` for direct udfs ([#1673](https://github.com/googleapis/python-bigquery-dataframes/issues/1673)) ([1a658b2](https://github.com/googleapis/python-bigquery-dataframes/commit/1a658b2aa43c4a7a7f2007a509b0e1401f925dab)) +* Publish bigframes blob(Multimodal) to preview ([#1693](https://github.com/googleapis/python-bigquery-dataframes/issues/1693)) ([e4c85ba](https://github.com/googleapis/python-bigquery-dataframes/commit/e4c85ba4813469d39edd7352201aefc26642d14c)) +* Support () operator between timedeltas ([#1702](https://github.com/googleapis/python-bigquery-dataframes/issues/1702)) ([edaac89](https://github.com/googleapis/python-bigquery-dataframes/commit/edaac89c03db1ffc93b56275c765d8a964f7d02d)) +* Support forecast_limit_lower_bound and forecast_limit_upper_bound in ARIMA_PLUS (and ARIMA_PLUS_XREG) models ([#1305](https://github.com/googleapis/python-bigquery-dataframes/issues/1305)) ([b16740e](https://github.com/googleapis/python-bigquery-dataframes/commit/b16740ef4ad7b1fbf731595238cf087c93c93066)) +* Support to_strip parameter for str.strip, str.lstrip and str.rstrip ([#1705](https://github.com/googleapis/python-bigquery-dataframes/issues/1705)) ([a84ee75](https://github.com/googleapis/python-bigquery-dataframes/commit/a84ee75ddd4d9dae1463e505549d74eb4f819338)) + + +### Bug Fixes + +* Fix dayofyear doc test ([#1701](https://github.com/googleapis/python-bigquery-dataframes/issues/1701)) ([9b777a0](https://github.com/googleapis/python-bigquery-dataframes/commit/9b777a019aa31a115a22289f21c7cd9df07aa8b9)) +* Fix issues with chunked arrow data ([#1700](https://github.com/googleapis/python-bigquery-dataframes/issues/1700)) ([e3289b7](https://github.com/googleapis/python-bigquery-dataframes/commit/e3289b7a64ee1400c6cb78e75cff4759d8da8b7a)) +* Rename columns with protected names such as `_TABLE_SUFFIX` in `to_gbq()` ([#1691](https://github.com/googleapis/python-bigquery-dataframes/issues/1691)) ([8ec6079](https://github.com/googleapis/python-bigquery-dataframes/commit/8ec607986fd38f357746fbaeabef2ce7ab3e501f)) + + +### Performance Improvements + +* Defer query in `read_gbq` with wildcard tables ([#1661](https://github.com/googleapis/python-bigquery-dataframes/issues/1661)) ([5c125c9](https://github.com/googleapis/python-bigquery-dataframes/commit/5c125c99d4632c617425c2ef5c399d17878c0043)) +* Rechunk result pages client side ([#1680](https://github.com/googleapis/python-bigquery-dataframes/issues/1680)) ([67d8760](https://github.com/googleapis/python-bigquery-dataframes/commit/67d876076027b6123e49d1d8ddee4e45eaa28f5d)) + + +### Dependencies + +* Move bigtable and pubsub to extras ([#1696](https://github.com/googleapis/python-bigquery-dataframes/issues/1696)) ([597d817](https://github.com/googleapis/python-bigquery-dataframes/commit/597d8178048b203cea4777f29b1ce95de7b0670e)) + + +### Documentation + +* Add snippets for Matrix Factorization tutorials ([#1630](https://github.com/googleapis/python-bigquery-dataframes/issues/1630)) ([24b37ae](https://github.com/googleapis/python-bigquery-dataframes/commit/24b37aece60460aabecce306397eb1bf6686f8a7)) +* Deprecate `bpd.options.bigquery.allow_large_results` in favor of `bpd.options.compute.allow_large_results` ([#1597](https://github.com/googleapis/python-bigquery-dataframes/issues/1597)) ([18780b4](https://github.com/googleapis/python-bigquery-dataframes/commit/18780b48a17dba2b3b3542500f027ae9527f6bee)) +* Include import statement in the bigframes code snippet ([#1699](https://github.com/googleapis/python-bigquery-dataframes/issues/1699)) ([08d70b6](https://github.com/googleapis/python-bigquery-dataframes/commit/08d70b6ad3ab3ac7b9a57d93da00168a8de7df9a)) +* Include the clean-up step in the udf code snippet ([#1698](https://github.com/googleapis/python-bigquery-dataframes/issues/1698)) ([48992e2](https://github.com/googleapis/python-bigquery-dataframes/commit/48992e26d460832704401bd2a3eedb800c5061cc)) +* Move multimodal notebook out of experimental folder ([#1712](https://github.com/googleapis/python-bigquery-dataframes/issues/1712)) ([68b6532](https://github.com/googleapis/python-bigquery-dataframes/commit/68b6532a780d6349a4b65994b696c8026457eb94)) +* Update blob_display option in snippets ([#1714](https://github.com/googleapis/python-bigquery-dataframes/issues/1714)) ([8b30143](https://github.com/googleapis/python-bigquery-dataframes/commit/8b30143e3320a730df168b5a72e6d18e631135ee)) + ## [2.3.0](https://github.com/googleapis/python-bigquery-dataframes/compare/v2.2.0...v2.3.0) (2025-05-06) diff --git a/bigframes/version.py b/bigframes/version.py index 3058b5f7a3..3a34d3d7bb 100644 --- a/bigframes/version.py +++ b/bigframes/version.py @@ -12,8 +12,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.3.0" +__version__ = "2.4.0" # {x-release-please-start-date} -__release_date__ = "2025-05-06" +__release_date__ = "2025-05-12" # {x-release-please-end} diff --git a/third_party/bigframes_vendored/version.py b/third_party/bigframes_vendored/version.py index 3058b5f7a3..3a34d3d7bb 100644 --- a/third_party/bigframes_vendored/version.py +++ b/third_party/bigframes_vendored/version.py @@ -12,8 +12,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.3.0" +__version__ = "2.4.0" # {x-release-please-start-date} -__release_date__ = "2025-05-06" +__release_date__ = "2025-05-12" # {x-release-please-end}