public boolean isTableEmpty(TableReference tableReference) throws IOException, InterruptedException { return bqServices.getDatasetService(bqOptions).isTableEmpty(tableReference); }
public void createTable(Table table) throws IOException, InterruptedException { bqServices.getDatasetService(bqOptions).createTable(table); }
public long insertAll(TableReference ref, List<TableRow> rowList) throws IOException, InterruptedException { List<ValueInSingleWindow<TableRow>> rows = rowList.stream() .map(r -> ValueInSingleWindow.of( r, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)) .collect(Collectors.toList()); return bqServices.getDatasetService(bqOptions) .insertAll(ref, rows, null, InsertRetryPolicy.alwaysRetry(), new ArrayList<>(), ErrorContainer.TABLE_ROW_ERROR_CONTAINER, false, false); }
/** Writes the accumulated rows into BigQuery with streaming API. */ private void flushRows( TableReference tableReference, List<ValueInSingleWindow<TableRow>> tableRows, List<String> uniqueIds, BigQueryOptions options, List<ValueInSingleWindow<ErrorT>> failedInserts) throws InterruptedException { if (!tableRows.isEmpty()) { try { long totalBytes = bqServices .getDatasetService(options) .insertAll( tableReference, tableRows, uniqueIds, retryPolicy, failedInserts, errorContainer, skipInvalidRows, ignoreUnknownValues); byteCounter.inc(totalBytes); } catch (IOException e) { throw new RuntimeException(e); } } } }
@Override public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws Exception { if (tableSizeBytes.get() == null) { TableReference table = setDefaultProjectIfAbsent( options.as(BigQueryOptions.class), BigQueryIO.JSON_FACTORY.fromString(jsonTable.get(), TableReference.class)); Table tableRef = bqServices.getDatasetService(options.as(BigQueryOptions.class)).getTable(table); Long numBytes = tableRef.getNumBytes(); if (tableRef.getStreamingBuffer() != null) { numBytes += tableRef.getStreamingBuffer().getEstimatedBytes().longValue(); } tableSizeBytes.compareAndSet(null, numBytes); } return tableSizeBytes.get(); }
@Override protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { TableReference tableToRemove = createTempTableReference( bqOptions.getProject(), createJobIdToken(bqOptions.getJobName(), stepUuid)); DatasetService tableService = bqServices.getDatasetService(bqOptions); LOG.info("Deleting temporary table with query results {}", tableToRemove); tableService.deleteTable(tableToRemove); LOG.info("Deleting temporary dataset with query results {}", tableToRemove.getDatasetId()); tableService.deleteDataset(tableToRemove.getProjectId(), tableToRemove.getDatasetId()); }
private PendingJobData startWriteRename( TableDestination finalTableDestination, Iterable<String> tempTableNames, ProcessContext c) throws Exception { WriteDisposition writeDisposition = (c.pane().getIndex() == 0) ? firstPaneWriteDisposition : WriteDisposition.WRITE_APPEND; CreateDisposition createDisposition = (c.pane().getIndex() == 0) ? firstPaneCreateDisposition : CreateDisposition.CREATE_NEVER; List<TableReference> tempTables = StreamSupport.stream(tempTableNames.spliterator(), false) .map(table -> BigQueryHelpers.fromJsonString(table, TableReference.class)) .collect(Collectors.toList()); ; // Make sure each destination table gets a unique job id. String jobIdPrefix = BigQueryHelpers.createJobId( c.sideInput(jobIdToken), finalTableDestination, -1, c.pane().getIndex()); BigQueryHelpers.PendingJob retryJob = startCopy( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), jobIdPrefix, finalTableDestination.getTableReference(), tempTables, writeDisposition, createDisposition); return new PendingJobData(retryJob, finalTableDestination, tempTables); }
DatasetService tableService = bqServices.getDatasetService(bqOptions); if (location == null) {
protected ExtractResult extractFiles(PipelineOptions options) throws Exception { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); TableReference tableToExtract = getTableToExtract(bqOptions); BigQueryServices.DatasetService datasetService = bqServices.getDatasetService(bqOptions); Table table = datasetService.getTable(tableToExtract); if (table == null) { throw new IOException( String.format( "Cannot start an export job since table %s does not exist", BigQueryHelpers.toTableSpec(tableToExtract))); } TableSchema schema = table.getSchema(); JobService jobService = bqServices.getJobService(bqOptions); String extractJobId = getExtractJobId(createJobIdToken(options.getJobName(), stepUuid)); final String extractDestinationDir = resolveTempLocation(bqOptions.getTempLocation(), "BigQueryExtractTemp", stepUuid); String bqLocation = BigQueryHelpers.getDatasetLocation( datasetService, tableToExtract.getProjectId(), tableToExtract.getDatasetId()); List<ResourceId> tempFiles = executeExtract( extractJobId, tableToExtract, jobService, bqOptions.getProject(), extractDestinationDir, bqLocation); return new ExtractResult(schema, tempFiles); }
DatasetService datasetService = getBigQueryServices().getDatasetService(bqOptions); BigQueryHelpers.verifyDatasetPresence(datasetService, table.get()); BigQueryHelpers.verifyTablePresence(datasetService, table.get());
@Override public void validate(PipelineOptions pipelineOptions) { BigQueryOptions options = pipelineOptions.as(BigQueryOptions.class); // The user specified a table. if (getJsonTableRef() != null && getJsonTableRef().isAccessible() && getValidate()) { TableReference table = getTableWithDefaultProject(options).get(); DatasetService datasetService = getBigQueryServices().getDatasetService(options); // Check for destination table presence and emptiness for early failure notification. // Note that a presence check can fail when the table or dataset is created by an earlier // stage of the pipeline. For these cases the #withoutValidation method can be used to // disable the check. BigQueryHelpers.verifyDatasetPresence(datasetService, table); if (getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) { BigQueryHelpers.verifyTablePresence(datasetService, table); } if (getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) { BigQueryHelpers.verifyTableNotExistOrEmpty(datasetService, table); } } }
@FinishBundle public void finishBundle(FinishBundleContext c) throws Exception { DatasetService datasetService = bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); PendingJobManager jobManager = new PendingJobManager(); for (PendingJobData pendingJob : pendingJobs) { jobManager.addPendingJob( pendingJob.retryJob, j -> { try { if (pendingJob.tableDestination.getTableDescription() != null) { TableReference ref = pendingJob.tableDestination.getTableReference(); datasetService.patchTableDescription( ref.clone() .setTableId(BigQueryHelpers.stripPartitionDecorator(ref.getTableId())), pendingJob.tableDestination.getTableDescription()); } removeTemporaryTables(datasetService, pendingJob.tempTables); return null; } catch (IOException | InterruptedException e) { return e; } }); } jobManager.waitForDone(); }