|
| 1 | +import logging |
| 2 | +import os |
| 3 | +import tempfile |
| 4 | +import time |
| 5 | +import uuid |
| 6 | +from typing import Any, Dict, List, Optional |
| 7 | + |
| 8 | +import pyarrow.parquet as pq |
| 9 | + |
| 10 | +from ray.data._internal.execution.interfaces import TaskContext |
| 11 | +from ray.data._internal.util import _check_import |
| 12 | +from ray.data.block import Block, BlockAccessor, BlockMetadata |
| 13 | +from ray.data.datasource.datasource import Datasource, Reader, ReadTask, WriteResult |
| 14 | +from ray.types import ObjectRef |
| 15 | +from ray.util.annotations import PublicAPI |
| 16 | + |
| 17 | +logger = logging.getLogger(__name__) |
| 18 | + |
| 19 | +MAX_RETRY_CNT = 10 |
| 20 | +RATE_LIMIT_EXCEEDED_SLEEP_TIME = 11 |
| 21 | + |
| 22 | + |
| 23 | +class _BigQueryDatasourceReader(Reader): |
| 24 | + def __init__( |
| 25 | + self, |
| 26 | + project_id: str, |
| 27 | + dataset: Optional[str] = None, |
| 28 | + query: Optional[str] = None, |
| 29 | + parallelism: Optional[int] = -1, |
| 30 | + **kwargs: Optional[Dict[str, Any]], |
| 31 | + ): |
| 32 | + self._project_id = project_id |
| 33 | + self._dataset = dataset |
| 34 | + self._query = query |
| 35 | + self._kwargs = kwargs |
| 36 | + |
| 37 | + if query is not None and dataset is not None: |
| 38 | + raise ValueError( |
| 39 | + "Query and dataset kwargs cannot both be provided " |
| 40 | + + "(must be mutually exclusive)." |
| 41 | + ) |
| 42 | + |
| 43 | + def get_read_tasks(self, parallelism: int) -> List[ReadTask]: |
| 44 | + from google.cloud import bigquery, bigquery_storage |
| 45 | + |
| 46 | + def _read_single_partition(stream) -> Block: |
| 47 | + client = bigquery_storage.BigQueryReadClient() |
| 48 | + reader = client.read_rows(stream.name) |
| 49 | + return reader.to_arrow() |
| 50 | + |
| 51 | + if self._query: |
| 52 | + query_client = bigquery.Client(project=self._project_id) |
| 53 | + query_job = query_client.query(self._query) |
| 54 | + query_job.result() |
| 55 | + destination = str(query_job.destination) |
| 56 | + dataset_id = destination.split(".")[-2] |
| 57 | + table_id = destination.split(".")[-1] |
| 58 | + else: |
| 59 | + self._validate_dataset_table_exist(self._project_id, self._dataset) |
| 60 | + dataset_id = self._dataset.split(".")[0] |
| 61 | + table_id = self._dataset.split(".")[1] |
| 62 | + |
| 63 | + bqs_client = bigquery_storage.BigQueryReadClient() |
| 64 | + table = f"projects/{self._project_id}/datasets/{dataset_id}/tables/{table_id}" |
| 65 | + |
| 66 | + if parallelism == -1: |
| 67 | + parallelism = None |
| 68 | + requested_session = bigquery_storage.types.ReadSession( |
| 69 | + table=table, |
| 70 | + data_format=bigquery_storage.types.DataFormat.ARROW, |
| 71 | + ) |
| 72 | + read_session = bqs_client.create_read_session( |
| 73 | + parent=f"projects/{self._project_id}", |
| 74 | + read_session=requested_session, |
| 75 | + max_stream_count=parallelism, |
| 76 | + ) |
| 77 | + |
| 78 | + read_tasks = [] |
| 79 | + logger.info("Created streams: " + str(len(read_session.streams))) |
| 80 | + if len(read_session.streams) < parallelism: |
| 81 | + logger.info( |
| 82 | + "The number of streams created by the " |
| 83 | + + "BigQuery Storage Read API is less than the requested " |
| 84 | + + "parallelism due to the size of the dataset." |
| 85 | + ) |
| 86 | + |
| 87 | + for stream in read_session.streams: |
| 88 | + # Create a metadata block object to store schema, etc. |
| 89 | + metadata = BlockMetadata( |
| 90 | + num_rows=None, |
| 91 | + size_bytes=None, |
| 92 | + schema=None, |
| 93 | + input_files=None, |
| 94 | + exec_stats=None, |
| 95 | + ) |
| 96 | + |
| 97 | + # Create the read task and pass the no-arg wrapper and metadata in |
| 98 | + read_task = ReadTask( |
| 99 | + lambda stream=stream: [_read_single_partition(stream)], |
| 100 | + metadata, |
| 101 | + ) |
| 102 | + read_tasks.append(read_task) |
| 103 | + |
| 104 | + return read_tasks |
| 105 | + |
| 106 | + def estimate_inmemory_data_size(self) -> Optional[int]: |
| 107 | + return None |
| 108 | + |
| 109 | + def _validate_dataset_table_exist(self, project_id: str, dataset: str) -> None: |
| 110 | + from google.api_core import exceptions |
| 111 | + from google.cloud import bigquery |
| 112 | + |
| 113 | + client = bigquery.Client(project=project_id) |
| 114 | + dataset_id = dataset.split(".")[0] |
| 115 | + try: |
| 116 | + client.get_dataset(dataset_id) |
| 117 | + except exceptions.NotFound: |
| 118 | + raise ValueError( |
| 119 | + "Dataset {} is not found. Please ensure that it exists.".format( |
| 120 | + dataset_id |
| 121 | + ) |
| 122 | + ) |
| 123 | + |
| 124 | + try: |
| 125 | + client.get_table(dataset) |
| 126 | + except exceptions.NotFound: |
| 127 | + raise ValueError( |
| 128 | + "Table {} is not found. Please ensure that it exists.".format(dataset) |
| 129 | + ) |
| 130 | + |
| 131 | + |
| 132 | +@PublicAPI(stability="alpha") |
| 133 | +class BigQueryDatasource(Datasource): |
| 134 | + def create_reader(self, **kwargs) -> Reader: |
| 135 | + _check_import(self, module="google.cloud", package="bigquery") |
| 136 | + _check_import(self, module="google.cloud", package="bigquery_storage") |
| 137 | + _check_import(self, module="google.api_core", package="exceptions") |
| 138 | + return _BigQueryDatasourceReader(**kwargs) |
| 139 | + |
| 140 | + def write( |
| 141 | + self, |
| 142 | + blocks: List[ObjectRef[Block]], |
| 143 | + ctx: TaskContext, |
| 144 | + project_id: str, |
| 145 | + dataset: str, |
| 146 | + ) -> WriteResult: |
| 147 | + from google.api_core import exceptions |
| 148 | + from google.cloud import bigquery |
| 149 | + |
| 150 | + def _write_single_block(block: Block, project_id: str, dataset: str): |
| 151 | + block = BlockAccessor.for_block(block).to_arrow() |
| 152 | + |
| 153 | + client = bigquery.Client(project=project_id) |
| 154 | + job_config = bigquery.LoadJobConfig(autodetect=True) |
| 155 | + job_config.source_format = bigquery.SourceFormat.PARQUET |
| 156 | + job_config.write_disposition = bigquery.WriteDisposition.WRITE_APPEND |
| 157 | + |
| 158 | + with tempfile.TemporaryDirectory() as temp_dir: |
| 159 | + fp = os.path.join(temp_dir, f"block_{uuid.uuid4()}.parquet") |
| 160 | + pq.write_table(block, fp, compression="SNAPPY") |
| 161 | + |
| 162 | + retry_cnt = 0 |
| 163 | + while retry_cnt < MAX_RETRY_CNT: |
| 164 | + with open(fp, "rb") as source_file: |
| 165 | + job = client.load_table_from_file( |
| 166 | + source_file, dataset, job_config=job_config |
| 167 | + ) |
| 168 | + retry_cnt += 1 |
| 169 | + try: |
| 170 | + logger.info(job.result()) |
| 171 | + break |
| 172 | + except exceptions.Forbidden as e: |
| 173 | + logger.info("Rate limit exceeded... Sleeping to try again") |
| 174 | + logger.debug(e) |
| 175 | + time.sleep(RATE_LIMIT_EXCEEDED_SLEEP_TIME) |
| 176 | + |
| 177 | + # Set up datasets to write |
| 178 | + client = bigquery.Client(project=project_id) |
| 179 | + dataset_id = dataset.split(".", 1)[0] |
| 180 | + try: |
| 181 | + client.create_dataset(f"{project_id}.{dataset_id}", timeout=30) |
| 182 | + logger.info("Created dataset " + dataset_id) |
| 183 | + except exceptions.Conflict: |
| 184 | + logger.info( |
| 185 | + f"Dataset {dataset_id} already exists. " |
| 186 | + "The table will be overwritten if it already exists." |
| 187 | + ) |
| 188 | + |
| 189 | + # Delete table if it already exists |
| 190 | + client.delete_table(f"{project_id}.{dataset}", not_found_ok=True) |
| 191 | + |
| 192 | + for block in blocks: |
| 193 | + _write_single_block(block, project_id, dataset) |
| 194 | + return "ok" |
0 commit comments