def _dataset_variable_factory(table_proxy, table_schema, select_cols, exemplar_row, orders, chunks, array_prefix): """ Returns a dictionary of dask arrays representing a series of getcols on the appropriate table. Produces variables for inclusion in a Dataset. Parameters ---------- table_proxy : :class:`daskms.table_proxy.TableProxy` Table proxy object table_schema : dict Table schema select_cols : list of strings List of columns to return exemplar_row : int row id used to possibly extract an exemplar array in order to determine the column shape and dtype attributes orders : tuple of :class:`dask.array.Array` A (sorted_rows, row_runs) tuple, specifying the appropriate rows to extract from the table. chunks : dict Chunking strategy for the dataset. array_prefix : str dask array string prefix Returns ------- dict A dictionary looking like :code:`{column: (arrays, dims)}`. """ sorted_rows, row_runs = orders dataset_vars = {"ROWID": (("row", ), sorted_rows)} for column in select_cols: try: meta = column_metadata(column, table_proxy, table_schema, chunks, exemplar_row) except ColumnMetadataError as e: exc_info = logging.DEBUG >= log.getEffectiveLevel() log.warning("Ignoring '%s': %s", column, e, exc_info=exc_info) continue full_dims = ("row", ) + meta.dims args = [row_runs, ("row", )] # We only need to pass in dimension extent arrays if # there is more than one chunk in any of the non-row columns. # In that case, we can getcol, otherwise getcolslice is required if not all(len(c) == 1 for c in meta.chunks): for d, c in zip(meta.dims, meta.chunks): # Create an array describing the dimension chunk extents args.append(dim_extents_array(d, c)) args.append((d, )) new_axes = {} else: # We need to inform blockwise about the size of our # new dimensions as no arrays with them are supplied new_axes = {d: s for d, s in zip(meta.dims, meta.shape)} # Add other variables args.extend([ table_proxy, None, column, None, meta.shape, None, meta.dtype, None ]) # Name of the dask array representing this column token = dask.base.tokenize(args) name = "-".join((array_prefix, column, token)) # Construct the array dask_array = da.blockwise(getter_wrapper, full_dims, *args, name=name, new_axes=new_axes, dtype=meta.dtype) dask_array = inlined_array(dask_array) # Assign into variable and dimension dataset dataset_vars[column] = (full_dims, dask_array) return dataset_vars
def _write_datasets(table, table_proxy, datasets, columns, descriptor, table_keywords, column_keywords): _, table_name, subtable = table_path_split(table) table_name = '::'.join((table_name, subtable)) if subtable else table_name row_orders = [] # Put table and column keywords table_proxy.submit(_put_keywords, WRITELOCK, table_keywords, column_keywords).result() # Sort datasets on (not has "ROWID", index) such that # datasets with ROWID's are handled first, while # those without (which imply appends to the MS) # are handled last sorted_datasets = sorted(enumerate(datasets), key=lambda t: ("ROWID" not in t[1].data_vars, t[0])) # Establish row orders for each dataset for di, ds in sorted_datasets: try: rowid = ds.ROWID.data except AttributeError: # Add operation # No ROWID's, assume they're missing from the table # and remaining datasets. Generate addrows # NOTE(sjperkins) # This could be somewhat brittle, but exists to # update MS empty subtables once they've been # created along with the main MS by a call to default_ms. # Users could also it to append rows to an existing table. # An xds_append_to_table may be a better solution... last_datasets = datasets[di:] last_row_orders = add_row_order_factory(table_proxy, last_datasets) # We don't inline the row ordering if it is derived # from the row sizes of provided arrays. # The range of possible dependencies are far too large to inline row_orders.extend([(False, lro) for lro in last_row_orders]) # We have established row orders for all datasets # at this point, quit the loop break else: # Update operation # Generate row orderings from existing row IDs row_order = rowid.map_blocks(row_run_factory, sort_dir="write", dtype=np.object) # TODO(sjperkins) # There's an assumption here that rowid is an # operation with minimal dependencies # (i.e. derived from xds_from_{ms, table}) # Caching flattens the graph into a single layer if len(row_order.__dask_graph__().layers) > 1: log.warning("Caching an update row ordering " "with more than one layer") row_order = cached_array(row_order) # Inline the row ordering in the graph row_orders.append((True, row_order)) assert len(row_orders) == len(datasets) datasets = [] for (di, ds), (inline, row_order) in zip(sorted_datasets, row_orders): # Hold the variables representing array writes write_vars = {} # Generate a dask array for each column for column in columns: try: variable = ds.data_vars[column] except KeyError: log.warning("Ignoring '%s' not present " "on dataset %d" % (column, di)) continue else: full_dims = variable.dims array = variable.data if not isinstance(array, da.Array): raise TypeError("%s on dataset %d is not a dask Array " "but a %s" % (column, di, type(array))) args = [row_order, ("row", )] # We only need to pass in dimension extent arrays if # there is more than one chunk in any of the non-row columns. # In that case, we can putcol, otherwise putcolslice is required if not all(len(c) == 1 for c in array.chunks[1:]): # Add extent arrays for d, c in zip(full_dims[1:], array.chunks[1:]): args.append(dim_extents_array(d, c)) args.append((d, )) # Add other variables args.extend([table_proxy, None, column, None, array, full_dims]) # Name of the dask array representing this column token = dask.base.tokenize(di, args) name = "-".join((table_name, 'write', column, token)) write_col = da.blockwise( putter_wrapper, full_dims, *args, # All dims shrink to 1, # a single bool is returned adjust_chunks={d: 1 for d in full_dims}, name=name, align_arrays=False, dtype=np.bool) if inline: write_col = inlined_array(write_col, [row_order]) write_vars[column] = (full_dims, write_col) # Append a dataset with the write operations datasets.append(Dataset(write_vars)) # Return an empty dataset if len(datasets) == 0: return Dataset({}) # Return singleton elif len(datasets) == 1: return datasets[0] return datasets
def _write_datasets(table, table_proxy, datasets, columns, descriptor, table_keywords, column_keywords): _, table_name, subtable = table_path_split(table) table_name = '::'.join((table_name, subtable)) if subtable else table_name writes = [] row_orders = [] # Put table and column keywords table_proxy.submit(_put_keywords, WRITELOCK, table_keywords, column_keywords).result() # Sort datasets on (not has "ROWID", index) such that # datasets with ROWID's are handled first, while # those without (which imply appends to the MS) # are handled last sorted_datasets = sorted(enumerate(datasets), key=lambda t: ("ROWID" not in t[1].data_vars, t[0])) # Establish row orders for each dataset for di, ds in sorted_datasets: try: rowid = ds.ROWID.data except AttributeError: # No ROWID's, assume they're missing from the table # and remaining datasets. Generate addrows # NOTE(sjperkins) # This could be somewhat brittle, but exists to # update of MS subtables once they've been # created (empty) along with the main MS by a call to default_ms. # Users could also it to append rows to an existing table. # An xds_append_to_table is probably the correct solution... last_datasets = datasets[di:] last_row_orders = add_row_order_factory(table_proxy, last_datasets) row_orders.extend(last_row_orders) # We have established row orders for all datasets # at this point, quit the loop break else: # Generate row orderings from existing row IDs row_order = rowid.map_blocks(row_run_factory, sort_dir="write", dtype=np.object) row_orders.append(row_order) assert len(row_orders) == len(datasets) for (di, ds), row_order in zip(sorted_datasets, row_orders): data_vars = ds.data_vars # Generate a dask array for each column for column in columns: try: variable = data_vars[column] except KeyError: log.warning("Ignoring '%s' not present " "on dataset %d" % (column, di)) continue else: full_dims = variable.dims array = variable.data args = [row_order, ("row", )] # We only need to pass in dimension extent arrays if # there is more than one chunk in any of the non-row columns. # In that case, we can putcol, otherwise putcolslice is required if not all(len(c) == 1 for c in array.chunks[1:]): # Add extent arrays for d, c in zip(full_dims[1:], array.chunks[1:]): args.append(dim_extents_array(d, c)) args.append((d, )) # Add other variables args.extend([table_proxy, None, column, None, array, full_dims]) # Name of the dask array representing this column token = dask.base.tokenize(di, args) name = "-".join((table_name, 'write', column, token)) write_col = da.blockwise( putter_wrapper, full_dims, *args, # All dims shrink to 1, # a single bool is returned adjust_chunks={d: 1 for d in full_dims}, name=name, align_arrays=False, dtype=np.bool) writes.append(write_col.ravel()) if len(writes) == 0: return da.full(1, True, dtype=np.bool) return da.concatenate(writes)