/** * Converts a JSON string to a {@link TableRow} object. If the data fails to convert, a {@link * RuntimeException} will be thrown. * * @param json The JSON string to parse. * @return The parsed {@link TableRow} object. */ private static TableRow convertJsonToTableRow(String json) { TableRow row; // Parse the JSON into a {@link TableRow} object. try (InputStream inputStream = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8))) { row = TableRowJsonCoder.of().decode(inputStream, Context.OUTER); } catch (IOException e) { throw new RuntimeException("Failed to serialize json to table row: " + json, e); } return row; }
Read() { this(BigQueryIO.read(TableRowParser.INSTANCE).withCoder(TableRowJsonCoder.of())); }
/** * Like {@link #read(SerializableFunction)} but represents each row as a {@link TableRow}. * * <p>This method is more convenient to use in some cases, but usually has significantly lower * performance than using {@link #read(SerializableFunction)} directly to parse data into a * domain-specific type, due to the overhead of converting the rows to {@link TableRow}. */ public static TypedRead<TableRow> readTableRows() { return read(new TableRowParser()).withCoder(TableRowJsonCoder.of()); }
private List<TableRow> readRows(String filename) throws IOException { Coder<TableRow> coder = TableRowJsonCoder.of(); List<TableRow> tableRows = Lists.newArrayList(); try (BufferedReader reader = Files.newBufferedReader(Paths.get(filename), StandardCharsets.UTF_8)) { String line; while ((line = reader.readLine()) != null) { TableRow tableRow = coder.decode( new ByteArrayInputStream(line.getBytes(StandardCharsets.UTF_8)), Context.OUTER); tableRows.add(tableRow); } } return tableRows; }
@Override public List<CoderProvider> getCoderProviders() { return ImmutableList.of( CoderProviders.forCoder(TypeDescriptor.of(TableRow.class), TableRowJsonCoder.of()), CoderProviders.forCoder(TypeDescriptor.of(TableRowInfo.class), TableRowInfoCoder.of())); } }
@Override protected long getEncodedElementByteSize(BigQueryInsertError value) throws Exception { String errorStrValue = MAPPER.writeValueAsString(value.getError()); String tableStrValue = MAPPER.writeValueAsString(value.getTable()); return StringUtf8Coder.of().getEncodedElementByteSize(errorStrValue) + TableRowJsonCoder.of().getEncodedElementByteSize(value.getRow()) + StringUtf8Coder.of().getEncodedElementByteSize(tableStrValue); }
@Override public BigQueryInsertError decode(InputStream inStream) throws IOException { TableDataInsertAllResponse.InsertErrors err = MAPPER.readValue( StringUtf8Coder.of().decode(inStream), TableDataInsertAllResponse.InsertErrors.class); TableRow row = TableRowJsonCoder.of().decode(inStream); TableReference ref = BigQueryHelpers.parseTableSpec(StringUtf8Coder.of().decode(inStream)); return new BigQueryInsertError(row, err, ref); }
static String encodeQuery(List<TableRow> rows) throws IOException { ListCoder<TableRow> listCoder = ListCoder.of(TableRowJsonCoder.of()); ByteArrayOutputStream output = new ByteArrayOutputStream(); listCoder.encode(rows, output, Context.OUTER); return Base64.encodeBase64String(output.toByteArray()); }
@Override public void encode(BigQueryInsertError value, OutputStream outStream) throws IOException { String errorStrValue = MAPPER.writeValueAsString(value.getError()); StringUtf8Coder.of().encode(errorStrValue, outStream); TableRowJsonCoder.of().encode(value.getRow(), outStream); StringUtf8Coder.of().encode(BigQueryHelpers.toTableSpec(value.getTable()), outStream); }
static List<TableRow> rowsFromEncodedQuery(String query) throws IOException { ListCoder<TableRow> listCoder = ListCoder.of(TableRowJsonCoder.of()); ByteArrayInputStream input = new ByteArrayInputStream(Base64.decodeBase64(query)); List<TableRow> rows = listCoder.decode(input, Context.OUTER); for (TableRow row : rows) { convertNumbers(row); } return rows; }
@Test public void testCoder_nullCell() throws CoderException { TableRow row = new TableRow(); row.set("temperature", Data.nullOf(Object.class)); row.set("max_temperature", Data.nullOf(Object.class)); byte[] bytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), row); TableRow newRow = CoderUtils.decodeFromByteArray(TableRowJsonCoder.of(), bytes); byte[] newBytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), newRow); Assert.assertArrayEquals(bytes, newBytes); }
@Override public WriteResult expand(PCollection<KV<TableDestination, TableRow>> input) { if (extendedErrorInfo) { TupleTag<BigQueryInsertError> failedInsertsTag = new TupleTag<>(FAILED_INSERTS_TAG_ID); PCollection<BigQueryInsertError> failedInserts = writeAndGetErrors( input, failedInsertsTag, BigQueryInsertErrorCoder.of(), ErrorContainer.BIG_QUERY_INSERT_ERROR_ERROR_CONTAINER); return WriteResult.withExtendedErrors(input.getPipeline(), failedInsertsTag, failedInserts); } else { TupleTag<TableRow> failedInsertsTag = new TupleTag<>(FAILED_INSERTS_TAG_ID); PCollection<TableRow> failedInserts = writeAndGetErrors( input, failedInsertsTag, TableRowJsonCoder.of(), ErrorContainer.TABLE_ROW_ERROR_CONTAINER); return WriteResult.in(input.getPipeline(), failedInsertsTag, failedInserts); } }
@Test public void testWriteValidateFailsCreateNoSchema() { p.enableAbandonedNodeEnforcement(false); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("no schema was provided"); p.apply(Create.empty(TableRowJsonCoder.of())) .apply( BigQueryIO.writeTableRows() .to("dataset.table") .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)); }
@Test public void testWriteEmptyPCollection() throws Exception { TableSchema schema = new TableSchema() .setFields( ImmutableList.of(new TableFieldSchema().setName("number").setType("INTEGER"))); p.apply(Create.empty(TableRowJsonCoder.of())) .apply( BigQueryIO.writeTableRows() .to("project-id:dataset-id.table-id") .withTestServices(fakeBqServices) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withSchema(schema) .withoutValidation()); p.run(); checkNotNull( fakeDatasetService.getTable( BigQueryHelpers.parseTableSpec("project-id:dataset-id.table-id"))); }
@Test public void testRuntimeOptionsNotCalledInApplyOutput() { p.enableAbandonedNodeEnforcement(false); BigQueryIO.Write<TableRow> write = BigQueryIO.writeTableRows() .to(p.newProvider("some-table")) .withSchema( ValueProvider.NestedValueProvider.of( p.newProvider("some-schema"), new BigQueryHelpers.JsonSchemaToTableSchema())) .withoutValidation(); p.apply(Create.empty(TableRowJsonCoder.of())).apply(write); // Test that this doesn't throw. DisplayData.from(write); }
@Test public void testWriteWithMissingSchemaFromView() throws Exception { PCollectionView<Map<String, String>> view = p.apply("Create schema view", Create.of(KV.of("foo", "bar"), KV.of("bar", "boo"))) .apply(View.asMap()); p.apply(Create.empty(TableRowJsonCoder.of())) .apply( BigQueryIO.writeTableRows() .to("dataset-id.table-id") .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withSchemaFromView(view) .withTestServices(fakeBqServices) .withoutValidation()); thrown.expectMessage("does not contain data for table destination dataset-id.table-id"); p.run(); }
@Test public void testCreateNeverWithStreaming() throws Exception { p.enableAbandonedNodeEnforcement(false); TableReference tableRef = new TableReference(); tableRef.setDatasetId("dataset"); tableRef.setTableId("sometable"); PCollection<TableRow> tableRows = p.apply(GenerateSequence.from(0)) .apply( MapElements.via( new SimpleFunction<Long, TableRow>() { @Override public TableRow apply(Long input) { return null; } })) .setCoder(TableRowJsonCoder.of()); tableRows.apply( BigQueryIO.writeTableRows() .to(tableRef) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_NEVER) .withoutValidation()); }
@Test public void testWriteUnknown() throws Exception { p.apply( Create.of( new TableRow().set("name", "a").set("number", 1), new TableRow().set("name", "b").set("number", 2), new TableRow().set("name", "c").set("number", 3)) .withCoder(TableRowJsonCoder.of())) .apply( BigQueryIO.writeTableRows() .to("project-id:dataset-id.table-id") .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_NEVER) .withTestServices(fakeBqServices) .withoutValidation()); thrown.expect(RuntimeException.class); thrown.expectMessage("Failed to create job"); p.run(); }
@Test public void testWriteFailedJobs() throws Exception { p.apply( Create.of( new TableRow().set("name", "a").set("number", 1), new TableRow().set("name", "b").set("number", 2), new TableRow().set("name", "c").set("number", 3)) .withCoder(TableRowJsonCoder.of())) .apply( BigQueryIO.writeTableRows() .to("dataset-id.table-id") .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_NEVER) .withTestServices(fakeBqServices) .withoutValidation()); thrown.expect(RuntimeException.class); thrown.expectMessage("Failed to create job with prefix"); thrown.expectMessage("reached max retries"); thrown.expectMessage("last failed job"); p.run(); }
private void runBigQueryToTablePipeline() { Pipeline p = Pipeline.create(options); BigQueryIO.Read bigQueryRead = BigQueryIO.read().fromQuery(options.getQuery()); if (options.getUsingStandardSql()) { bigQueryRead = bigQueryRead.usingStandardSql(); } PCollection<TableRow> input = p.apply(bigQueryRead); if (options.getReshuffle()) { input = input .apply(WithKeys.<Void, TableRow>of((Void) null)) .setCoder(KvCoder.of(VoidCoder.of(), TableRowJsonCoder.of())) .apply(Reshuffle.<Void, TableRow>of()) .apply(Values.<TableRow>create()); } input.apply( BigQueryIO.writeTableRows() .to(options.getOutput()) .withSchema(options.getOutputSchema()) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)); p.run().waitUntilFinish(); }